From 94011d91ba29e92f3dcd43fd77814af869fc4800 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 9 Jun 2021 02:03:36 +0000 Subject: [PATCH 001/950] initial implement --- src/CMakeLists.txt | 2 + src/Storages/FileLog/Buffer_fwd.h | 10 + .../FileLog/FileLogBlockInputStream.cpp | 173 ++++++++++ .../FileLog/FileLogBlockInputStream.h | 49 +++ .../FileLog/ReadBufferFromFileLog.cpp | 156 +++++++++ src/Storages/FileLog/ReadBufferFromFileLog.h | 95 +++++ src/Storages/FileLog/StorageFileLog.cpp | 326 ++++++++++++++++++ src/Storages/FileLog/StorageFileLog.h | 92 +++++ src/Storages/registerStorages.cpp | 6 +- 9 files changed, 908 insertions(+), 1 deletion(-) create mode 100644 src/Storages/FileLog/Buffer_fwd.h create mode 100644 src/Storages/FileLog/FileLogBlockInputStream.cpp create mode 100644 src/Storages/FileLog/FileLogBlockInputStream.h create mode 100644 src/Storages/FileLog/ReadBufferFromFileLog.cpp create mode 100644 src/Storages/FileLog/ReadBufferFromFileLog.h create mode 100644 src/Storages/FileLog/StorageFileLog.cpp create mode 100644 src/Storages/FileLog/StorageFileLog.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 272bea4f6d7..b82c72d19f4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,6 +104,8 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() +add_headers_and_sources(dbms Storages/FileLog) + list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/src/Storages/FileLog/Buffer_fwd.h b/src/Storages/FileLog/Buffer_fwd.h new file mode 100644 index 00000000000..ec644aa7d36 --- /dev/null +++ b/src/Storages/FileLog/Buffer_fwd.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ +class ReadBufferFromFileLog; + +using ReadBufferFromFileLogPtr = std::shared_ptr; +} diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogBlockInputStream.cpp new file mode 100644 index 00000000000..538fbb6049a --- /dev/null +++ b/src/Storages/FileLog/FileLogBlockInputStream.cpp @@ -0,0 +1,173 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +// with default poll timeout (500ms) it will give about 5 sec delay for doing 10 retries +// when selecting from empty topic +const auto MAX_FAILED_POLL_ATTEMPTS = 10; + +FileLogBlockInputStream::FileLogBlockInputStream( + StorageFileLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_, + const Names & columns, + Poco::Logger * log_, + size_t max_block_size_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) + , column_names(columns) + , log(log_) + , max_block_size(max_block_size_) + , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) + , virtual_header( + metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) +{ +} + +Block FileLogBlockInputStream::getHeader() const +{ + return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); +} + +void FileLogBlockInputStream::readPrefixImpl() +{ + buffer = storage.getBuffer(); + + if (!buffer) + return; + + buffer->open(); +} + +Block FileLogBlockInputStream::readImpl() +{ + if (!buffer || finished) + return Block(); + + finished = true; + // now it's one-time usage InputStream + // one block of the needed size (or with desired flush timeout) is formed in one internal iteration + // otherwise external iteration will reuse that and logic will became even more fuzzy + MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); + // MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + + InputPort port(input_format->getPort().getHeader(), input_format.get()); + connect(input_format->getPort(), port); + port.setNeeded(); + + std::optional exception_message; + auto read_file_log = [&] { + size_t new_rows = 0; + while (true) + { + auto status = input_format->prepare(); + + switch (status) + { + case IProcessor::Status::Ready: + input_format->work(); + break; + + case IProcessor::Status::Finished: + input_format->resetParser(); + return new_rows; + + case IProcessor::Status::PortFull: + { + auto chunk = port.pull(); + + // that was returning bad value before https://github.com/ClickHouse/ClickHouse/pull/8005 + // if will be backported should go together with #8005 + auto chunk_rows = chunk.getNumRows(); + new_rows += chunk_rows; + + auto columns = chunk.detachColumns(); + for (size_t i = 0, s = columns.size(); i < s; ++i) + { + result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + } + break; + } + case IProcessor::Status::NeedData: + case IProcessor::Status::Async: + case IProcessor::Status::ExpandPipeline: + throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); + } + } + }; + + size_t total_rows = 0; + + while (true) + { + size_t new_rows = 0; + exception_message.reset(); + if (buffer->poll()) + { + try + { + new_rows = read_file_log(); + } + catch (Exception & e) + { + throw; + } + } + if (new_rows) + { + total_rows = total_rows + new_rows; + } + + if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) + { + break; + } + } + + if (total_rows == 0) + return Block(); + + /// MATERIALIZED columns can be added here, but I think + // they are not needed here: + // and it's misleading to use them here, + // as columns 'materialized' that way stays 'ephemeral' + // i.e. will not be stored anythere + // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. + + auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); + // auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + // for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + // result_block.insert(column); + + return ConvertingBlockInputStream( + std::make_shared(result_block), + getHeader(), + ConvertingBlockInputStream::MatchColumnsMode::Name) + .read(); +} + +void FileLogBlockInputStream::readSuffixImpl() +{ + if (buffer) + buffer->close(); +} + +} diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogBlockInputStream.h new file mode 100644 index 00000000000..9a625e8499d --- /dev/null +++ b/src/Storages/FileLog/FileLogBlockInputStream.h @@ -0,0 +1,49 @@ +#pragma once + +#include + +#include +#include + + +namespace Poco +{ + class Logger; +} +namespace DB +{ +class FileLogBlockInputStream : public IBlockInputStream +{ +public: + FileLogBlockInputStream( + StorageFileLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_, + const Names & columns, + Poco::Logger * log_, + size_t max_block_size_); + ~FileLogBlockInputStream() override = default; + + String getName() const override { return storage.getName(); } + Block getHeader() const override; + + void readPrefixImpl() override; + Block readImpl() override; + void readSuffixImpl() override; + +private: + StorageFileLog & storage; + StorageMetadataPtr metadata_snapshot; + ContextPtr context; + Names column_names; + Poco::Logger * log; + UInt64 max_block_size; + + ReadBufferFromFileLogPtr buffer; + bool finished = false; + + const Block non_virtual_header; + const Block virtual_header; +}; + +} diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp new file mode 100644 index 00000000000..f2dabdc2b83 --- /dev/null +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -0,0 +1,156 @@ +#include +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMMIT_OFFSET; +} + +using namespace std::chrono_literals; + +ReadBufferFromFileLog::ReadBufferFromFileLog( + const std::vector & log_files_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_) + : ReadBuffer(nullptr, 0) + , log(log_) + , batch_size(max_batch_size) + , poll_timeout(poll_timeout_) + , context(context_) + , log_files(log_files_.begin(), log_files_.end()) +{ +} + +void ReadBufferFromFileLog::open() +{ + for (const auto & file : log_files) + file_status[file].reader = std::ifstream(file); + + wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); + wait_task->deactivate(); + + select_task = context->getMessageBrokerSchedulePool().createTask("selectTask", [this] { selectFunc(); }); + select_task->activateAndSchedule(); + + cleanUnprocessed(); + allowed = false; +} + +void ReadBufferFromFileLog::cleanUnprocessed() +{ + records.clear(); + current = records.begin(); + BufferBase::set(nullptr, 0, 0); +} + +void ReadBufferFromFileLog::close() +{ + wait_task->deactivate(); + select_task->deactivate(); + + for (auto & status : file_status) + status.second.reader.close(); +} + +// it do the poll when needed +bool ReadBufferFromFileLog::poll() +{ + + if (hasMorePolledRecords()) + { + allowed = true; + return true; + } + + auto new_records = pollBatch(batch_size); + if (new_records.empty()) + { + LOG_TRACE(log, "No records returned"); + return false; + } + else + { + records = std::move(new_records); + current = records.begin(); + LOG_TRACE(log, "Polled batch of {} records. ", records.size()); + allowed = true; + return true; + } + + +} + +ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_size_) +{ + Records new_records; + new_records.reserve(batch_size_); + + readNewRecords(new_records, batch_size); + if (new_records.size() == batch_size_) + return new_records; + + wait_task->activateAndSchedule(); + while (!time_out && new_records.size() != batch_size_) + { + readNewRecords(new_records, batch_size); + } + + wait_task->deactivate(); + time_out = false; + return new_records; +} + +void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) +{ + size_t need_records_size = batch_size_ - new_records.size(); + size_t read_records_size = 0; + for (auto & status : file_status) + { + while (read_records_size < need_records_size && status.second.reader.good() && !status.second.reader.eof()) + { + Record record; + std::getline(status.second.reader, record); + new_records.emplace_back(record); + ++read_records_size; + } + if (read_records_size == need_records_size) + break; + } +} + +bool ReadBufferFromFileLog::nextImpl() +{ + if (!allowed || !hasMorePolledRecords()) + return false; + + auto * new_position = const_cast(current->data()); + BufferBase::set(new_position, current->size(), 0); + allowed = false; + + ++current; + + return true; +} + +void ReadBufferFromFileLog::waitFunc() +{ + sleepForMicroseconds(poll_timeout); + time_out = true; +} + +void ReadBufferFromFileLog::selectFunc() +{ +} +} + diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h new file mode 100644 index 00000000000..e6a635fcaeb --- /dev/null +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -0,0 +1,95 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include +#include + +namespace Poco +{ + class Logger; +} + +namespace DB +{ +class ReadBufferFromFileLog : public ReadBuffer +{ +public: + ReadBufferFromFileLog( + const std::vector & log_files_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_); + + ~ReadBufferFromFileLog() override = default; + + void open(); + void close(); + + auto pollTimeout() const { return poll_timeout; } + + inline bool hasMorePolledRecords() const { return current != records.end(); } + + bool poll(); + +private: + enum class FileStatus + { + BEGIN, + NO_CHANGE, + UPDATED, + REMOVED + }; + + struct FileContext + { + std::mutex status_mutex; + FileStatus status = FileStatus::BEGIN; + std::ifstream reader; + }; + + Poco::Logger * log; + const size_t batch_size = 1; + const size_t poll_timeout = 0; + + bool time_out = false; + + using NameToFile = std::unordered_map; + NameToFile file_status; + + ContextPtr context; + + bool allowed = true; + + const std::vector log_files; + + using Record = std::string; + using Records = std::vector; + + Records records; + Records::const_iterator current; + + using TaskThread = BackgroundSchedulePool::TaskHolder; + + TaskThread wait_task; + TaskThread select_task; + + Records pollBatch(size_t batch_size_); + + void readNewRecords(Records & new_records, size_t batch_size_); + + // void drain(); + void cleanUnprocessed(); + + bool nextImpl() override; + + void waitFunc(); + + void selectFunc(); +}; +} diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp new file mode 100644 index 00000000000..33026430e99 --- /dev/null +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -0,0 +1,326 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + const auto RESCHEDULE_MS = 500; + const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) +} + +StorageFileLog::StorageFileLog( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & path_, + const String & format_name_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , path(path_) + , format_name(format_name_) + , log(&Poco::Logger::get("StorageFile (" + table_id_.table_name + ")")) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); + + Poco::File file(path); + + if (file.isFile()) + { + log_files.emplace_back(path); + } + else if (file.isDirectory()) + { + Poco::DirectoryIterator dir_iter(file); + Poco::DirectoryIterator end; + while (dir_iter != end) + { + if (dir_iter->isFile()) + log_files.emplace_back(dir_iter->path()); + ++dir_iter; + } + } + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + thread->deactivate(); + task = std::make_shared(std::move(thread)); +} + +Pipe StorageFileLog::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) +{ + auto modified_context = Context::createCopy(local_context); + + return Pipe(std::make_shared( + std::make_shared(*this, metadata_snapshot, modified_context, column_names, log, 1))); + +} + +void StorageFileLog::startup() +{ + try + { + createReadBuffer(); + } + catch (const Exception &) + { + tryLogCurrentException(log); + } + + task->holder->activateAndSchedule(); +} + + +void StorageFileLog::shutdown() +{ + task->stream_cancelled = true; + + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + + LOG_TRACE(log, "Closing files"); + destroyReadBuffer(); +} + +size_t StorageFileLog::getMaxBlockSize() const +{ + return getContext()->getSettingsRef().max_insert_block_size.value; +} + +size_t StorageFileLog::getPollMaxBatchSize() const +{ + size_t batch_size = getContext()->getSettingsRef().max_block_size.value; + + return std::min(batch_size,getMaxBlockSize()); +} + +size_t StorageFileLog::getPollTimeoutMillisecond() const +{ + return getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); +} + + +bool StorageFileLog::checkDependencies(const StorageID & table_id) +{ + // Check if all dependencies are attached + auto dependencies = DatabaseCatalog::instance().getDependencies(table_id); + if (dependencies.empty()) + return true; + + // Check the dependencies are ready? + for (const auto & db_tab : dependencies) + { + auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); + if (!table) + return false; + + // If it materialized view, check it's target table + auto * materialized_view = dynamic_cast(table.get()); + if (materialized_view && !materialized_view->tryGetTargetTable()) + return false; + + // Check all its dependencies + if (!checkDependencies(db_tab)) + return false; + } + + return true; +} + +void StorageFileLog::createReadBuffer() +{ + auto new_context = Context::createCopy(getContext()); + buffer = std::make_shared(log_files, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); +} + +void StorageFileLog::destroyReadBuffer() +{ + if (buffer) + buffer->close(); +} + +void StorageFileLog::threadFunc() +{ + try + { + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + if (dependencies_count) + { + auto start_time = std::chrono::steady_clock::now(); + + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!task->stream_cancelled) + { + if (!checkDependencies(table_id)) + break; + + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + // Exit the loop & reschedule if some stream stalled + auto some_stream_is_stalled = streamToViews(); + if (some_stream_is_stalled) + { + LOG_TRACE(log, "Stream(s) stalled. Reschedule."); + break; + } + + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts-start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + break; + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + // Wait for attached views + if (!task->stream_cancelled) + task->holder->scheduleAfter(RESCHEDULE_MS); +} + + +bool StorageFileLog::streamToViews() +{ + Stopwatch watch; + + auto table_id = getStorageID(); + auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); + if (!table) + throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); + auto metadata_snapshot = getInMemoryMetadataPtr(); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + size_t block_size = getMaxBlockSize(); + + auto new_context = Context::createCopy(getContext()); + + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, new_context, false, true, true); + auto block_io = interpreter.execute(); + + auto stream = std::make_shared( + *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), log, block_size); + + // Limit read batch to maximum block size to allow DDL + StreamLocalLimits limits; + + limits.speed_limits.max_execution_time = getContext()->getSettingsRef().stream_flush_interval_ms; + + limits.timeout_overflow_mode = OverflowMode::BREAK; + stream->setLimits(limits); + + // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. + // It will be cancelled on underlying layer (kafka buffer) + std::atomic stub = {false}; + size_t rows = 0; + copyData( + *stream, *block_io.out, [&rows](const Block & block) { rows += block.rows(); }, &stub); + + UInt64 milliseconds = watch.elapsedMilliseconds(); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", + formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); + + return true; +} + +void registerStorageFileLog(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + size_t args_count = engine_args.size(); + + if (args_count != 2) + throw Exception("Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::BAD_ARGUMENTS); + + auto path_ast = evaluateConstantExpressionAsLiteral(engine_args[0], args.getContext()); + auto format_ast = evaluateConstantExpressionAsLiteral(engine_args[1], args.getContext()); + + auto path = path_ast->as().value.safeGet(); + auto format = format_ast->as().value.safeGet(); + + return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format); + }; + + factory.registerStorage( + "FileLog", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + }); +} + +NamesAndTypesList StorageFileLog::getVirtuals() const +{ + auto result = NamesAndTypesList{}; + return result; +} + +Names StorageFileLog::getVirtualColumnNames() +{ + auto result = Names{}; + return result; +} + +} diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h new file mode 100644 index 00000000000..9cb7556ea05 --- /dev/null +++ b/src/Storages/FileLog/StorageFileLog.h @@ -0,0 +1,92 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ +class StorageFileLog final : public ext::shared_ptr_helper, public IStorage, WithContext +{ + friend struct ext::shared_ptr_helper; + +public: + using Files = std::vector; + + std::string getName() const override { return "FileLog"; } + + bool noPushingToViews() const override { return true; } + + void startup() override; + void shutdown() override; + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + const auto & getFormatName() const { return format_name; } + + NamesAndTypesList getVirtuals() const override; + static Names getVirtualColumnNames(); + + auto & getBuffer() { return buffer; } + +protected: + StorageFileLog( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & path_, + const String & format_name_); + +private: + const String path; + const String format_name; + Poco::Logger * log; + + Files log_files; + + ReadBufferFromFileLogPtr buffer; + + std::mutex mutex; + + // Stream thread + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::shared_ptr task; + + void createReadBuffer(); + void destroyReadBuffer(); + + void threadFunc(); + + size_t getPollMaxBatchSize() const; + size_t getMaxBlockSize() const; + size_t getPollTimeoutMillisecond() const; + + bool streamToViews(); + bool checkDependencies(const StorageID & table_id); +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 0b302ee437a..7d32139ca64 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -50,6 +50,8 @@ void registerStorageMongoDB(StorageFactory & factory); void registerStorageKafka(StorageFactory & factory); #endif +void registerStorageFileLog(StorageFactory & factory); + #if USE_AMQPCPP void registerStorageRabbitMQ(StorageFactory & factory); #endif @@ -112,7 +114,9 @@ void registerStorages() registerStorageKafka(factory); #endif - #if USE_AMQPCPP + registerStorageFileLog(factory); + +#if USE_AMQPCPP registerStorageRabbitMQ(factory); #endif From b15099fe6fb241b1a83069350a787f90ae0b7e3e Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 3 Jul 2021 17:14:56 +0000 Subject: [PATCH 002/950] add directory watcher --- src/CMakeLists.txt | 6 +- src/Core/config_core.h.in | 1 + .../FileLog/FileLogBlockInputStream.cpp | 27 +----- .../FileLog/FileLogDirectoryWatcher.cpp | 89 +++++++++++++++++ .../FileLog/FileLogDirectoryWatcher.h | 48 ++++++++++ .../FileLog/ReadBufferFromFileLog.cpp | 96 +++++++++++++++---- src/Storages/FileLog/ReadBufferFromFileLog.h | 18 ++-- src/Storages/FileLog/StorageFileLog.cpp | 34 +------ src/Storages/FileLog/StorageFileLog.h | 2 - ...StorageSystemBuildOptions.generated.cpp.in | 2 + src/Storages/registerStorages.cpp | 8 +- 11 files changed, 243 insertions(+), 88 deletions(-) create mode 100644 src/Storages/FileLog/FileLogDirectoryWatcher.cpp create mode 100644 src/Storages/FileLog/FileLogDirectoryWatcher.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b82c72d19f4..c05a39e7239 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,7 +104,11 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() -add_headers_and_sources(dbms Storages/FileLog) +set (USE_FILELOG 1) + +if(USE_FILELOG) + add_headers_and_sources(dbms Storages/FileLog) +endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index e250e013913..a1d7f2b8d0d 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -15,3 +15,4 @@ #cmakedefine01 USE_LIBPQXX #cmakedefine01 USE_NURAFT #cmakedefine01 USE_KRB5 +#cmakedefine01 USE_FILELOG diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogBlockInputStream.cpp index 538fbb6049a..eb82c891fd6 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.cpp +++ b/src/Storages/FileLog/FileLogBlockInputStream.cpp @@ -15,10 +15,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -// with default poll timeout (500ms) it will give about 5 sec delay for doing 10 retries -// when selecting from empty topic -const auto MAX_FAILED_POLL_ATTEMPTS = 10; - FileLogBlockInputStream::FileLogBlockInputStream( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -55,15 +51,10 @@ void FileLogBlockInputStream::readPrefixImpl() Block FileLogBlockInputStream::readImpl() { - if (!buffer || finished) + if (!buffer) return Block(); - finished = true; - // now it's one-time usage InputStream - // one block of the needed size (or with desired flush timeout) is formed in one internal iteration - // otherwise external iteration will reuse that and logic will became even more fuzzy MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); - // MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); @@ -93,8 +84,6 @@ Block FileLogBlockInputStream::readImpl() { auto chunk = port.pull(); - // that was returning bad value before https://github.com/ClickHouse/ClickHouse/pull/8005 - // if will be backported should go together with #8005 auto chunk_rows = chunk.getNumRows(); new_rows += chunk_rows; @@ -125,7 +114,7 @@ Block FileLogBlockInputStream::readImpl() { new_rows = read_file_log(); } - catch (Exception & e) + catch (Exception &) { throw; } @@ -133,6 +122,7 @@ Block FileLogBlockInputStream::readImpl() if (new_rows) { total_rows = total_rows + new_rows; + LOG_INFO(log, "FileLogBlockInputStream, {} rows data polled from buffer.", new_rows); } if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) @@ -144,18 +134,7 @@ Block FileLogBlockInputStream::readImpl() if (total_rows == 0) return Block(); - /// MATERIALIZED columns can be added here, but I think - // they are not needed here: - // and it's misleading to use them here, - // as columns 'materialized' that way stays 'ephemeral' - // i.e. will not be stored anythere - // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. - auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); - // auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - - // for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - // result_block.insert(column); return ConvertingBlockInputStream( std::make_shared(result_block), diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp new file mode 100644 index 00000000000..e95f7c55c4d --- /dev/null +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -0,0 +1,89 @@ +#include +#include +#include + +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) + : path(path_), dw(std::make_shared(path)) +{ + dw->itemAdded += Poco::delegate(this, &FileLogDirectoryWatcher::onItemAdded); + dw->itemRemoved += Poco::delegate(this, &FileLogDirectoryWatcher::onItemRemoved); + dw->itemModified += Poco::delegate(this, &FileLogDirectoryWatcher::onItemModified); + dw->itemMovedFrom += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedFrom); + dw->itemMovedTo += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedTo); +} + +FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() +{ + std::lock_guard lock(mutex); + Events res; + res.swap(events); + return res; +} + +bool FileLogDirectoryWatcher::getError() const +{ + return error; +} + +const std::string & FileLogDirectoryWatcher::getPath() const +{ + return path; +} + +void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemAdded"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + + +void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemRemoved"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + + +void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemModified"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + +void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemMovedFrom"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + +void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemMovedTo"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + + +void FileLogDirectoryWatcher::onError(const Poco::Exception &) +{ + error = true; +} diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h new file mode 100644 index 00000000000..b0fde72f57c --- /dev/null +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + +#include +#include + +class FileLogDirectoryWatcher +{ +public: + struct DirEvent + { + Poco::DirectoryWatcher::DirectoryEventType type; + std::string callback; + std::string path; + }; + + using Events = std::deque; + + explicit FileLogDirectoryWatcher(const std::string & path_); + ~FileLogDirectoryWatcher() = default; + + Events getEvents(); + + bool getError() const; + + const std::string & getPath() const; + +protected: + void onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onError(const Poco::Exception &); + +private: + const std::string path; + std::shared_ptr dw; + + std::mutex mutex; + + Events events; + + bool error = false; +}; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index f2dabdc2b83..1eedbee136a 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include #include @@ -15,33 +17,49 @@ namespace ErrorCodes extern const int CANNOT_COMMIT_OFFSET; } -using namespace std::chrono_literals; - ReadBufferFromFileLog::ReadBufferFromFileLog( - const std::vector & log_files_, - Poco::Logger * log_, - size_t max_batch_size, - size_t poll_timeout_, - ContextPtr context_) + const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_) : ReadBuffer(nullptr, 0) + , path(path_) , log(log_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , context(context_) - , log_files(log_files_.begin(), log_files_.end()) { } void ReadBufferFromFileLog::open() { - for (const auto & file : log_files) - file_status[file].reader = std::ifstream(file); + Poco::File file(path); + + bool path_is_directory = false; + + if (file.isFile()) + { + file_status[path].reader = std::ifstream(path); + } + else if (file.isDirectory()) + { + path_is_directory = true; + Poco::DirectoryIterator dir_iter(file); + Poco::DirectoryIterator end; + while (dir_iter != end) + { + if (dir_iter->isFile()) + file_status[dir_iter->path()].reader = std::ifstream(dir_iter->path()); + ++dir_iter; + } + } wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); wait_task->deactivate(); - select_task = context->getMessageBrokerSchedulePool().createTask("selectTask", [this] { selectFunc(); }); - select_task->activateAndSchedule(); + if (path_is_directory) + { + FileLogDirectoryWatcher dw(path); + select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this, &dw] { watchFunc(dw); }); + select_task->activateAndSchedule(); + } cleanUnprocessed(); allowed = false; @@ -63,7 +81,6 @@ void ReadBufferFromFileLog::close() status.second.reader.close(); } -// it do the poll when needed bool ReadBufferFromFileLog::poll() { @@ -113,10 +130,19 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) { + std::lock_guard lock(status_mutex); + size_t need_records_size = batch_size_ - new_records.size(); size_t read_records_size = 0; + for (auto & status : file_status) { + if (status.second.status == FileStatus::NO_CHANGE) + continue; + + if (status.second.status == FileStatus::REMOVED) + file_status.erase(status.first); + while (read_records_size < need_records_size && status.second.reader.good() && !status.second.reader.eof()) { Record record; @@ -124,6 +150,11 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ new_records.emplace_back(record); ++read_records_size; } + + // Read to the end of the file + if (status.second.reader.eof()) + status.second.status = FileStatus::NO_CHANGE; + if (read_records_size == need_records_size) break; } @@ -149,8 +180,41 @@ void ReadBufferFromFileLog::waitFunc() time_out = true; } -void ReadBufferFromFileLog::selectFunc() +void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) { -} -} + while (true) + { + sleepForNanoseconds(poll_timeout); + auto error = dw.getError(); + if (error) + LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); + + auto events = dw.getEvents(); + std::lock_guard lock(status_mutex); + + for (const auto & event : events) + { + switch (event.type) + { + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + file_status[event.path].reader = std::ifstream(event.path); + break; + + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + file_status[event.path].status = FileStatus::UPDATED; + break; + + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + file_status[event.path].status = FileStatus::REMOVED; + break; + + default: + LOG_INFO(log, "Undefine event type"); + } + } + } +} +} diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index e6a635fcaeb..20b6b6d0c2d 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -19,12 +20,7 @@ namespace DB class ReadBufferFromFileLog : public ReadBuffer { public: - ReadBufferFromFileLog( - const std::vector & log_files_, - Poco::Logger * log_, - size_t max_batch_size, - size_t poll_timeout_, - ContextPtr context_); + ReadBufferFromFileLog(const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_); ~ReadBufferFromFileLog() override = default; @@ -48,11 +44,12 @@ private: struct FileContext { - std::mutex status_mutex; FileStatus status = FileStatus::BEGIN; std::ifstream reader; }; + const String path; + Poco::Logger * log; const size_t batch_size = 1; const size_t poll_timeout = 0; @@ -62,12 +59,12 @@ private: using NameToFile = std::unordered_map; NameToFile file_status; + std::mutex status_mutex; + ContextPtr context; bool allowed = true; - const std::vector log_files; - using Record = std::string; using Records = std::vector; @@ -83,13 +80,12 @@ private: void readNewRecords(Records & new_records, size_t batch_size_); - // void drain(); void cleanUnprocessed(); bool nextImpl() override; void waitFunc(); - void selectFunc(); + void watchFunc(FileLogDirectoryWatcher & dw); }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 33026430e99..7970feb2996 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -23,7 +23,6 @@ #include #include #include -#include #include #include #include @@ -67,24 +66,6 @@ StorageFileLog::StorageFileLog( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - Poco::File file(path); - - if (file.isFile()) - { - log_files.emplace_back(path); - } - else if (file.isDirectory()) - { - Poco::DirectoryIterator dir_iter(file); - Poco::DirectoryIterator end; - while (dir_iter != end) - { - if (dir_iter->isFile()) - log_files.emplace_back(dir_iter->path()); - ++dir_iter; - } - } - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); thread->deactivate(); task = std::make_shared(std::move(thread)); @@ -157,7 +138,6 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) if (dependencies.empty()) return true; - // Check the dependencies are ready? for (const auto & db_tab : dependencies) { auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); @@ -180,7 +160,7 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) void StorageFileLog::createReadBuffer() { auto new_context = Context::createCopy(getContext()); - buffer = std::make_shared(log_files, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); + buffer = std::make_shared(path, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); } void StorageFileLog::destroyReadBuffer() @@ -208,13 +188,7 @@ void StorageFileLog::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // Exit the loop & reschedule if some stream stalled - auto some_stream_is_stalled = streamToViews(); - if (some_stream_is_stalled) - { - LOG_TRACE(log, "Stream(s) stalled. Reschedule."); - break; - } + streamToViews(); auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts-start_time); @@ -255,14 +229,12 @@ bool StorageFileLog::streamToViews() auto new_context = Context::createCopy(getContext()); - // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); auto stream = std::make_shared( *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), log, block_size); - // Limit read batch to maximum block size to allow DDL StreamLocalLimits limits; limits.speed_limits.max_execution_time = getContext()->getSettingsRef().stream_flush_interval_ms; @@ -270,8 +242,6 @@ bool StorageFileLog::streamToViews() limits.timeout_overflow_mode = OverflowMode::BREAK; stream->setLimits(limits); - // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. - // It will be cancelled on underlying layer (kafka buffer) std::atomic stub = {false}; size_t rows = 0; copyData( diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 9cb7556ea05..dda759c7e38 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -59,8 +59,6 @@ private: const String format_name; Poco::Logger * log; - Files log_files; - ReadBufferFromFileLogPtr buffer; std::mutex mutex; diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 8fe574da643..18cab0700ec 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,6 +50,8 @@ const char * auto_config_build[] "USE_LDAP", "@USE_LDAP@", "TZDATA_VERSION", "@TZDATA_VERSION@", "USE_KRB5", "@USE_KRB5@", + "USE_KRB5", "@USE_KRB5@", + "USE_FILELOG", "@USE_FILELOG@", nullptr, nullptr }; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 7d32139ca64..863015ed828 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -50,8 +50,6 @@ void registerStorageMongoDB(StorageFactory & factory); void registerStorageKafka(StorageFactory & factory); #endif -void registerStorageFileLog(StorageFactory & factory); - #if USE_AMQPCPP void registerStorageRabbitMQ(StorageFactory & factory); #endif @@ -69,6 +67,10 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory); void registerStorageExternalDistributed(StorageFactory & factory); #endif +#if USE_FILELOG +void registerStorageFileLog(StorageFactory & factory); +#endif + void registerStorages() { auto & factory = StorageFactory::instance(); @@ -114,7 +116,9 @@ void registerStorages() registerStorageKafka(factory); #endif +#if USE_FILELOG registerStorageFileLog(factory); +#endif #if USE_AMQPCPP registerStorageRabbitMQ(factory); From eafe8a2e595cd25c5958e0c77dbd717f49d9da1d Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 3 Jul 2021 17:53:09 +0000 Subject: [PATCH 003/950] fix --- CMakeLists.txt | 2 ++ src/CMakeLists.txt | 2 -- src/Storages/FileLog/FileLogBlockInputStream.h | 1 - src/Storages/FileLog/ReadBufferFromFileLog.cpp | 3 --- src/Storages/FileLog/ReadBufferFromFileLog.h | 2 +- 5 files changed, 3 insertions(+), 7 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d23e5f540d3..8522a6ff42b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -553,6 +553,8 @@ include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) +set (USE_FILELOG ON) + set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index c05a39e7239..c906c730641 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,8 +104,6 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() -set (USE_FILELOG 1) - if(USE_FILELOG) add_headers_and_sources(dbms Storages/FileLog) endif() diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogBlockInputStream.h index 9a625e8499d..290d3cf07f8 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.h +++ b/src/Storages/FileLog/FileLogBlockInputStream.h @@ -40,7 +40,6 @@ private: UInt64 max_block_size; ReadBufferFromFileLogPtr buffer; - bool finished = false; const Block non_virtual_header; const Block virtual_header; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 1eedbee136a..171bc8b0668 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -210,9 +210,6 @@ void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: file_status[event.path].status = FileStatus::REMOVED; break; - - default: - LOG_INFO(log, "Undefine event type"); } } } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 20b6b6d0c2d..090b8581827 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -86,6 +86,6 @@ private: void waitFunc(); - void watchFunc(FileLogDirectoryWatcher & dw); + [[noreturn ]] void watchFunc(FileLogDirectoryWatcher & dw); }; } From 4be47ca7e75e75135225f1b5ada6b6e850f308ce Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 4 Jul 2021 06:16:40 +0000 Subject: [PATCH 004/950] update --- .../FileLog/FileLogBlockInputStream.cpp | 3 --- .../FileLog/FileLogBlockInputStream.h | 4 ++-- .../FileLog/ReadBufferFromFileLog.cpp | 19 +++++++++++++------ src/Storages/FileLog/ReadBufferFromFileLog.h | 13 ++++++++++++- src/Storages/FileLog/StorageFileLog.cpp | 18 +++++++++++++----- 5 files changed, 40 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogBlockInputStream.cpp index eb82c891fd6..5c206bfae6d 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.cpp +++ b/src/Storages/FileLog/FileLogBlockInputStream.cpp @@ -20,13 +20,11 @@ FileLogBlockInputStream::FileLogBlockInputStream( const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr & context_, const Names & columns, - Poco::Logger * log_, size_t max_block_size_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) - , log(log_) , max_block_size(max_block_size_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header( @@ -122,7 +120,6 @@ Block FileLogBlockInputStream::readImpl() if (new_rows) { total_rows = total_rows + new_rows; - LOG_INFO(log, "FileLogBlockInputStream, {} rows data polled from buffer.", new_rows); } if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogBlockInputStream.h index 290d3cf07f8..d72ab5649ee 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.h +++ b/src/Storages/FileLog/FileLogBlockInputStream.h @@ -20,7 +20,6 @@ public: const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr & context_, const Names & columns, - Poco::Logger * log_, size_t max_block_size_); ~FileLogBlockInputStream() override = default; @@ -31,12 +30,13 @@ public: Block readImpl() override; void readSuffixImpl() override; + bool isStalled() { return !buffer || buffer->isStalled(); } + private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; ContextPtr context; Names column_names; - Poco::Logger * log; UInt64 max_block_size; ReadBufferFromFileLogPtr buffer; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 171bc8b0668..1ba111be8f8 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -32,8 +32,6 @@ void ReadBufferFromFileLog::open() { Poco::File file(path); - bool path_is_directory = false; - if (file.isFile()) { file_status[path].reader = std::ifstream(path); @@ -56,8 +54,7 @@ void ReadBufferFromFileLog::open() if (path_is_directory) { - FileLogDirectoryWatcher dw(path); - select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this, &dw] { watchFunc(dw); }); + select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); select_task->activateAndSchedule(); } @@ -75,7 +72,9 @@ void ReadBufferFromFileLog::cleanUnprocessed() void ReadBufferFromFileLog::close() { wait_task->deactivate(); - select_task->deactivate(); + + if (path_is_directory) + select_task->deactivate(); for (auto & status : file_status) status.second.reader.close(); @@ -90,6 +89,8 @@ bool ReadBufferFromFileLog::poll() return true; } + buffer_status = BufferStatus::NO_RECORD_RETURNED; + auto new_records = pollBatch(batch_size); if (new_records.empty()) { @@ -100,7 +101,10 @@ bool ReadBufferFromFileLog::poll() { records = std::move(new_records); current = records.begin(); + LOG_TRACE(log, "Polled batch of {} records. ", records.size()); + + buffer_status = BufferStatus::NOT_STALLED; allowed = true; return true; } @@ -180,8 +184,9 @@ void ReadBufferFromFileLog::waitFunc() time_out = true; } -void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) +void ReadBufferFromFileLog::watchFunc() { + FileLogDirectoryWatcher dw(path); while (true) { sleepForNanoseconds(poll_timeout); @@ -197,6 +202,8 @@ void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) { switch (event.type) { + LOG_TRACE(log, "New event {} watched.", event.callback); + case Poco::DirectoryWatcher::DW_ITEM_ADDED: file_status[event.path].reader = std::ifstream(event.path); break; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 090b8581827..ce14acc9f67 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -33,7 +33,14 @@ public: bool poll(); + bool isStalled() { return buffer_status != BufferStatus::NOT_STALLED; } + private: + enum class BufferStatus + { + NO_RECORD_RETURNED, + NOT_STALLED, + }; enum class FileStatus { BEGIN, @@ -48,8 +55,12 @@ private: std::ifstream reader; }; + BufferStatus buffer_status; + const String path; + bool path_is_directory = false; + Poco::Logger * log; const size_t batch_size = 1; const size_t poll_timeout = 0; @@ -86,6 +97,6 @@ private: void waitFunc(); - [[noreturn ]] void watchFunc(FileLogDirectoryWatcher & dw); + [[noreturn]] void watchFunc(); }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7970feb2996..f6f1535c97e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -83,8 +83,7 @@ Pipe StorageFileLog::read( auto modified_context = Context::createCopy(local_context); return Pipe(std::make_shared( - std::make_shared(*this, metadata_snapshot, modified_context, column_names, log, 1))); - + std::make_shared(*this, metadata_snapshot, modified_context, column_names, 1))); } void StorageFileLog::startup() @@ -188,7 +187,12 @@ void StorageFileLog::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - streamToViews(); + auto stream_is_stalled = streamToViews(); + if (stream_is_stalled) + { + LOG_TRACE(log, "Stream stalled. Reschedule."); + break; + } auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts-start_time); @@ -233,7 +237,7 @@ bool StorageFileLog::streamToViews() auto block_io = interpreter.execute(); auto stream = std::make_shared( - *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), log, block_size); + *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), block_size); StreamLocalLimits limits; @@ -247,11 +251,15 @@ bool StorageFileLog::streamToViews() copyData( *stream, *block_io.out, [&rows](const Block & block) { rows += block.rows(); }, &stub); + bool stream_is_stalled = false; + + stream_is_stalled = stream->as()->isStalled(); + UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return true; + return stream_is_stalled; } void registerStorageFileLog(StorageFactory & factory) From f9a76666c9aca03278ec20988c1c23b90bc4f451 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 4 Jul 2021 06:31:41 +0000 Subject: [PATCH 005/950] fix fix fix fix --- .../FileLog/FileLogDirectoryWatcher.cpp | 49 +++++++++---------- .../FileLog/FileLogDirectoryWatcher.h | 21 ++++---- src/Storages/FileLog/StorageFileLog.cpp | 28 +++++------ ...StorageSystemBuildOptions.generated.cpp.in | 1 - 4 files changed, 48 insertions(+), 51 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index e95f7c55c4d..931f9c9aa06 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -34,56 +34,55 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director { std::lock_guard lock(mutex); DirEvent de; - de.callback = "onItemAdded"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + de.callback = "onItemAdded"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemRemoved"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + DirEvent de; + de.callback = "onItemRemoved"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemModified"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + DirEvent de; + de.callback = "onItemModified"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); DirEvent de; - de.callback = "onItemMovedFrom"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + de.callback = "onItemMovedFrom"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemMovedTo"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + DirEvent de; + de.callback = "onItemMovedTo"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } - void FileLogDirectoryWatcher::onError(const Poco::Exception &) { - error = true; + error = true; } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index b0fde72f57c..de15186a3b8 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,20 +4,19 @@ #include #include -#include #include class FileLogDirectoryWatcher { public: struct DirEvent - { - Poco::DirectoryWatcher::DirectoryEventType type; - std::string callback; - std::string path; - }; + { + Poco::DirectoryWatcher::DirectoryEventType type; + std::string callback; + std::string path; + }; - using Events = std::deque; + using Events = std::vector; explicit FileLogDirectoryWatcher(const std::string & path_); ~FileLogDirectoryWatcher() = default; @@ -30,10 +29,10 @@ public: protected: void onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent & ev); + void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent & ev); + void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent & ev); void onError(const Poco::Exception &); private: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index f6f1535c97e..a5e1b8731da 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -88,28 +88,28 @@ Pipe StorageFileLog::read( void StorageFileLog::startup() { - try - { - createReadBuffer(); - } - catch (const Exception &) - { - tryLogCurrentException(log); - } + try + { + createReadBuffer(); + } + catch (const Exception &) + { + tryLogCurrentException(log); + } - task->holder->activateAndSchedule(); + task->holder->activateAndSchedule(); } void StorageFileLog::shutdown() { - task->stream_cancelled = true; + task->stream_cancelled = true; - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); - LOG_TRACE(log, "Closing files"); - destroyReadBuffer(); + LOG_TRACE(log, "Closing files"); + destroyReadBuffer(); } size_t StorageFileLog::getMaxBlockSize() const diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 18cab0700ec..bb25342bd87 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,7 +50,6 @@ const char * auto_config_build[] "USE_LDAP", "@USE_LDAP@", "TZDATA_VERSION", "@TZDATA_VERSION@", "USE_KRB5", "@USE_KRB5@", - "USE_KRB5", "@USE_KRB5@", "USE_FILELOG", "@USE_FILELOG@", nullptr, nullptr From fdc256e30dab3a5817764d9a4a96580ecc2f1182 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 4 Jul 2021 08:52:05 +0000 Subject: [PATCH 006/950] fix build --- src/Storages/FileLog/StorageFileLog.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index dda759c7e38..93b67fe5a31 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include @@ -16,9 +16,9 @@ namespace DB { -class StorageFileLog final : public ext::shared_ptr_helper, public IStorage, WithContext +class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext { - friend struct ext::shared_ptr_helper; + friend struct shared_ptr_helper; public: using Files = std::vector; From 85aa71c377a783f115852bca7827460ec3328e27 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 11 Jul 2021 03:37:48 +0000 Subject: [PATCH 007/950] fix build --- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 1ba111be8f8..47985e32cb4 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -202,19 +202,21 @@ void ReadBufferFromFileLog::watchFunc() { switch (event.type) { - LOG_TRACE(log, "New event {} watched.", event.callback); case Poco::DirectoryWatcher::DW_ITEM_ADDED: + LOG_TRACE(log, "New event {} watched.", event.callback); file_status[event.path].reader = std::ifstream(event.path); break; case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + LOG_TRACE(log, "New event {} watched.", event.callback); file_status[event.path].status = FileStatus::UPDATED; break; case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + LOG_TRACE(log, "New event {} watched.", event.callback); file_status[event.path].status = FileStatus::REMOVED; break; } From 8dbb2a3ed9ad0db79ce6262593893cf68dc8c5e1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 11 Jul 2021 06:18:03 +0000 Subject: [PATCH 008/950] fix --- src/Storages/FileLog/FileLogDirectoryWatcher.h | 1 + src/Storages/FileLog/ReadBufferFromFileLog.cpp | 1 - src/Storages/FileLog/ReadBufferFromFileLog.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 5 ++--- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index de15186a3b8..99fd5cf0af1 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,6 +4,7 @@ #include #include +#include #include class FileLogDirectoryWatcher diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 47985e32cb4..e5849b50894 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_COMMIT_OFFSET; } ReadBufferFromFileLog::ReadBufferFromFileLog( diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index ce14acc9f67..e74c93d189e 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -25,7 +25,7 @@ public: ~ReadBufferFromFileLog() override = default; void open(); - void close(); + void close(); auto pollTimeout() const { return poll_timeout; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a5e1b8731da..565b206e2cc 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -38,9 +38,7 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -270,7 +268,8 @@ void registerStorageFileLog(StorageFactory & factory) size_t args_count = engine_args.size(); if (args_count != 2) - throw Exception("Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::BAD_ARGUMENTS); + throw Exception( + "Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto path_ast = evaluateConstantExpressionAsLiteral(engine_args[0], args.getContext()); auto format_ast = evaluateConstantExpressionAsLiteral(engine_args[1], args.getContext()); From a2256b1307ed1a567bbc21dd50935f737bd7c632 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 22 Aug 2021 01:27:22 +0800 Subject: [PATCH 009/950] Dedicated Mark/Uncompressed cache for skip indices --- programs/local/LocalServer.cpp | 11 ++++ programs/server/Server.cpp | 11 ++++ src/Interpreters/AsynchronousMetrics.cpp | 16 +++++ src/Interpreters/Context.cpp | 58 +++++++++++++++++++ src/Interpreters/Context.h | 10 ++++ src/Interpreters/InterpreterSystemQuery.cpp | 10 ++++ src/Parsers/ASTSystemQuery.cpp | 4 ++ src/Parsers/ASTSystemQuery.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 9 +++ .../MergeTree/MergeTreeDataSelectExecutor.h | 2 + .../MergeTree/MergeTreeIndexReader.cpp | 21 ++++++- src/Storages/MergeTree/MergeTreeIndexReader.h | 2 + 12 files changed, 153 insertions(+), 3 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 957bda4d75c..5e9b1570b66 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -262,6 +262,17 @@ try if (mark_cache_size) global_context->setMarkCache(mark_cache_size); + /// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. + size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0); + if (index_uncompressed_cache_size) + global_context->setIndexUncompressedCache(index_uncompressed_cache_size); + + /// Size of cache for index marks (index of MergeTree skip indices). It is necessary. + /// Specify default value for index_mark_cache_size explicitly! + size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", 0); + if (index_mark_cache_size) + global_context->setIndexMarkCache(index_mark_cache_size); + /// A cache for mmapped files. size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. if (mmap_cache_size) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7c1527bf4b0..55283e7cf46 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -942,6 +942,17 @@ if (ThreadFuzzer::instance().isEffective()) } global_context->setMarkCache(mark_cache_size); + /// Size of cache for uncompressed blocks of MergeTree indices. Zero means disabled. + size_t index_uncompressed_cache_size = config().getUInt64("index_uncompressed_cache_size", 0); + if (index_uncompressed_cache_size) + global_context->setIndexUncompressedCache(index_uncompressed_cache_size); + + /// Size of cache for index marks (index of MergeTree skip indices). It is necessary. + /// Specify default value for index_mark_cache_size explicitly! + size_t index_mark_cache_size = config().getUInt64("index_mark_cache_size", 0); + if (index_mark_cache_size) + global_context->setIndexMarkCache(index_mark_cache_size); + /// A cache for mmapped files. size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. if (mmap_cache_size) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index fd02aa4abec..19c45257846 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -546,6 +546,22 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti } } + { + if (auto index_mark_cache = getContext()->getIndexMarkCache()) + { + new_values["IndexMarkCacheBytes"] = index_mark_cache->weight(); + new_values["IndexMarkCacheFiles"] = index_mark_cache->count(); + } + } + + { + if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) + { + new_values["IndexUncompressedCacheBytes"] = index_uncompressed_cache->weight(); + new_values["IndexUncompressedCacheCells"] = index_uncompressed_cache->count(); + } + } + { if (auto mmap_cache = getContext()->getMMappedFileCache()) { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 583f59c20b5..e11d5eb1c14 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -184,6 +184,8 @@ struct ContextSharedPart std::unique_ptr access_control_manager; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. + mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. + mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices. mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads. ProcessList process_list; /// Executing queries at the moment. MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree) @@ -1420,6 +1422,56 @@ void Context::dropMarkCache() const } +void Context::setIndexUncompressedCache(size_t max_size_in_bytes) +{ + auto lock = getLock(); + + if (shared->index_uncompressed_cache) + throw Exception("Index uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR); + + shared->index_uncompressed_cache = std::make_shared(max_size_in_bytes); +} + + +UncompressedCachePtr Context::getIndexUncompressedCache() const +{ + auto lock = getLock(); + return shared->index_uncompressed_cache; +} + + +void Context::dropIndexUncompressedCache() const +{ + auto lock = getLock(); + if (shared->index_uncompressed_cache) + shared->index_uncompressed_cache->reset(); +} + + +void Context::setIndexMarkCache(size_t cache_size_in_bytes) +{ + auto lock = getLock(); + + if (shared->index_mark_cache) + throw Exception("Index mark cache has been already created.", ErrorCodes::LOGICAL_ERROR); + + shared->index_mark_cache = std::make_shared(cache_size_in_bytes); +} + +MarkCachePtr Context::getIndexMarkCache() const +{ + auto lock = getLock(); + return shared->index_mark_cache; +} + +void Context::dropIndexMarkCache() const +{ + auto lock = getLock(); + if (shared->index_mark_cache) + shared->index_mark_cache->reset(); +} + + void Context::setMMappedFileCache(size_t cache_size_in_num_entries) { auto lock = getLock(); @@ -1454,6 +1506,12 @@ void Context::dropCaches() const if (shared->mark_cache) shared->mark_cache->reset(); + if (shared->index_uncompressed_cache) + shared->index_uncompressed_cache->reset(); + + if (shared->index_mark_cache) + shared->index_mark_cache->reset(); + if (shared->mmap_cache) shared->mmap_cache->reset(); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1b636deb532..9928022b841 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -667,6 +667,16 @@ public: std::shared_ptr getMarkCache() const; void dropMarkCache() const; + /// Create a cache of index uncompressed blocks of specified size. This can be done only once. + void setIndexUncompressedCache(size_t max_size_in_bytes); + std::shared_ptr getIndexUncompressedCache() const; + void dropIndexUncompressedCache() const; + + /// Create a cache of index marks of specified size. This can be done only once. + void setIndexMarkCache(size_t cache_size_in_bytes); + std::shared_ptr getIndexMarkCache() const; + void dropIndexMarkCache() const; + /// Create a cache of mapped files to avoid frequent open/map/unmap/close and to reuse from several threads. void setMMappedFileCache(size_t cache_size_in_num_entries); std::shared_ptr getMMappedFileCache() const; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d4ac555add0..03a23e9eb35 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -277,6 +277,14 @@ BlockIO InterpreterSystemQuery::execute() getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); system_context->dropUncompressedCache(); break; + case Type::DROP_INDEX_MARK_CACHE: + getContext()->checkAccess(AccessType::SYSTEM_DROP_MARK_CACHE); + system_context->dropIndexMarkCache(); + break; + case Type::DROP_INDEX_UNCOMPRESSED_CACHE: + getContext()->checkAccess(AccessType::SYSTEM_DROP_UNCOMPRESSED_CACHE); + system_context->dropIndexUncompressedCache(); + break; case Type::DROP_MMAP_CACHE: getContext()->checkAccess(AccessType::SYSTEM_DROP_MMAP_CACHE); system_context->dropMMappedFileCache(); @@ -721,6 +729,8 @@ AccessRightsElements InterpreterSystemQuery::getRequiredAccessForDDLOnCluster() case Type::DROP_COMPILED_EXPRESSION_CACHE: [[fallthrough]]; #endif case Type::DROP_UNCOMPRESSED_CACHE: + case Type::DROP_INDEX_MARK_CACHE: + case Type::DROP_INDEX_UNCOMPRESSED_CACHE: { required_access.emplace_back(AccessType::SYSTEM_DROP_CACHE); break; diff --git a/src/Parsers/ASTSystemQuery.cpp b/src/Parsers/ASTSystemQuery.cpp index 5d01e124b0e..0f52df2f4ae 100644 --- a/src/Parsers/ASTSystemQuery.cpp +++ b/src/Parsers/ASTSystemQuery.cpp @@ -30,6 +30,10 @@ const char * ASTSystemQuery::typeToString(Type type) return "DROP MARK CACHE"; case Type::DROP_UNCOMPRESSED_CACHE: return "DROP UNCOMPRESSED CACHE"; + case Type::DROP_INDEX_MARK_CACHE: + return "DROP INDEX MARK CACHE"; + case Type::DROP_INDEX_UNCOMPRESSED_CACHE: + return "DROP INDEX UNCOMPRESSED CACHE"; case Type::DROP_MMAP_CACHE: return "DROP MMAP CACHE"; #if USE_EMBEDDED_COMPILER diff --git a/src/Parsers/ASTSystemQuery.h b/src/Parsers/ASTSystemQuery.h index fa7b6ece59a..c88fd4bf7ad 100644 --- a/src/Parsers/ASTSystemQuery.h +++ b/src/Parsers/ASTSystemQuery.h @@ -24,6 +24,8 @@ public: DROP_DNS_CACHE, DROP_MARK_CACHE, DROP_UNCOMPRESSED_CACHE, + DROP_INDEX_MARK_CACHE, + DROP_INDEX_UNCOMPRESSED_CACHE, DROP_MMAP_CACHE, #if USE_EMBEDDED_COMPILER DROP_COMPILED_EXPRESSION_CACHE, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f5c1890154a..b36b64d5651 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -834,6 +834,9 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf) leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf); + auto mark_cache = context->getIndexMarkCache(); + auto uncompressed_cache = context->getIndexUncompressedCache(); + auto process_part = [&](size_t part_index) { auto & part = parts[part_index]; @@ -870,6 +873,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd reader_settings, total_granules, granules_dropped, + mark_cache.get(), + uncompressed_cache.get(), log); index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed); @@ -1408,6 +1413,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( const MergeTreeReaderSettings & reader_settings, size_t & total_granules, size_t & granules_dropped, + MarkCache * mark_cache, + UncompressedCache * uncompressed_cache, Poco::Logger * log) { const std::string & path_prefix = part->getFullRelativePath() + index_helper->getFileName(); @@ -1433,6 +1440,8 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( index_helper, part, index_marks_count, ranges, + mark_cache, + uncompressed_cache, reader_settings); MarkRanges res; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 92c4382dc90..b114f498bf0 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -90,6 +90,8 @@ private: const MergeTreeReaderSettings & reader_settings, size_t & total_granules, size_t & granules_dropped, + MarkCache * mark_cache, + UncompressedCache * uncompressed_cache, Poco::Logger * log); struct PartFilterCounters diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 0a0f2511914..fd7358967f3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -11,13 +11,15 @@ std::unique_ptr makeIndexReader( MergeTreeData::DataPartPtr part, size_t marks_count, const MarkRanges & all_mark_ranges, + MarkCache * mark_cache, + UncompressedCache * uncompressed_cache, MergeTreeReaderSettings settings) { return std::make_unique( part->volume->getDisk(), part->getFullRelativePath() + index->getFileName(), extension, marks_count, all_mark_ranges, - std::move(settings), nullptr, nullptr, + std::move(settings), mark_cache, uncompressed_cache, part->getFileSizeOrZero(index->getFileName() + extension), &part->index_granularity_info, ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE); @@ -29,14 +31,27 @@ namespace DB { MergeTreeIndexReader::MergeTreeIndexReader( - MergeTreeIndexPtr index_, MergeTreeData::DataPartPtr part_, size_t marks_count_, const MarkRanges & all_mark_ranges_, + MergeTreeIndexPtr index_, + MergeTreeData::DataPartPtr part_, + size_t marks_count_, + const MarkRanges & all_mark_ranges_, + MarkCache * mark_cache, + UncompressedCache * uncompressed_cache, MergeTreeReaderSettings settings) : index(index_) { const std::string & path_prefix = part_->getFullRelativePath() + index->getFileName(); auto index_format = index->getDeserializedFormat(part_->volume->getDisk(), path_prefix); - stream = makeIndexReader(index_format.extension, index_, part_, marks_count_, all_mark_ranges_, std::move(settings)); + stream = makeIndexReader( + index_format.extension, + index_, + part_, + marks_count_, + all_mark_ranges_, + mark_cache, + uncompressed_cache, + std::move(settings)); version = index_format.version; stream->seekToStart(); diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.h b/src/Storages/MergeTree/MergeTreeIndexReader.h index 4facd43c175..799dae154bf 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.h +++ b/src/Storages/MergeTree/MergeTreeIndexReader.h @@ -16,6 +16,8 @@ public: MergeTreeData::DataPartPtr part_, size_t marks_count_, const MarkRanges & all_mark_ranges_, + MarkCache * mark_cache, + UncompressedCache * uncompressed_cache, MergeTreeReaderSettings settings); ~MergeTreeIndexReader(); From 595005eb211b60b548819963232de325312dd893 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 4 Sep 2021 17:04:35 +0000 Subject: [PATCH 010/950] refactor some code --- src/Storages/FileLog/FileLogSettings.cpp | 41 +++ src/Storages/FileLog/FileLogSettings.h | 36 +++ ...BlockInputStream.cpp => FileLogSource.cpp} | 56 ++-- ...eLogBlockInputStream.h => FileLogSource.h} | 33 +- .../FileLog/ReadBufferFromFileLog.cpp | 69 +---- src/Storages/FileLog/ReadBufferFromFileLog.h | 23 +- src/Storages/FileLog/StorageFileLog.cpp | 281 +++++++++++++----- src/Storages/FileLog/StorageFileLog.h | 39 ++- 8 files changed, 366 insertions(+), 212 deletions(-) create mode 100644 src/Storages/FileLog/FileLogSettings.cpp create mode 100644 src/Storages/FileLog/FileLogSettings.h rename src/Storages/FileLog/{FileLogBlockInputStream.cpp => FileLogSource.cpp} (73%) rename src/Storages/FileLog/{FileLogBlockInputStream.h => FileLogSource.h} (57%) diff --git a/src/Storages/FileLog/FileLogSettings.cpp b/src/Storages/FileLog/FileLogSettings.cpp new file mode 100644 index 00000000000..2cd42c35870 --- /dev/null +++ b/src/Storages/FileLog/FileLogSettings.cpp @@ -0,0 +1,41 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(FileLogSettingsTraits, LIST_OF_FILELOG_SETTINGS) + +void FileLogSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for storage " + storage_def.engine->name); + throw; + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h new file mode 100644 index 00000000000..cfea9d1e195 --- /dev/null +++ b/src/Storages/FileLog/FileLogSettings.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTStorage; + + +#define FILELOG_RELATED_SETTINGS(M) \ + /* default is stream_poll_timeout_ms */ \ + M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \ + /* default is min(max_block_size, kafka_max_block_size)*/ \ + M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ + /* default is = max_insert_block_size / kafka_num_consumers */ \ + M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ + M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0) + +#define LIST_OF_FILELOG_SETTINGS(M) \ + FILELOG_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(FileLogSettingsTraits, LIST_OF_FILELOG_SETTINGS) + + +/** Settings for the Kafka engine. + * Could be loaded from a CREATE TABLE query (SETTINGS clause). + */ +struct FileLogSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogSource.cpp similarity index 73% rename from src/Storages/FileLog/FileLogBlockInputStream.cpp rename to src/Storages/FileLog/FileLogSource.cpp index 5c206bfae6d..4a15cd07f25 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -1,11 +1,11 @@ -#include - #include #include #include #include #include +#include #include +#include #include namespace DB @@ -15,42 +15,34 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -FileLogBlockInputStream::FileLogBlockInputStream( +FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_, + const ContextPtr & context_, const Names & columns, - size_t max_block_size_) + size_t max_block_size_, + size_t poll_time_out_, + size_t stream_number_, + size_t max_streams_number_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) + , poll_time_out(poll_time_out_) + , stream_number(stream_number_) + , max_streams_number(max_streams_number_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { + createReadBuffer(); } -Block FileLogBlockInputStream::getHeader() const -{ - return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); -} - -void FileLogBlockInputStream::readPrefixImpl() -{ - buffer = storage.getBuffer(); - - if (!buffer) - return; - - buffer->open(); -} - -Block FileLogBlockInputStream::readImpl() +Chunk FileLogSource::generate() { if (!buffer) - return Block(); + return {}; MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); @@ -104,8 +96,8 @@ Block FileLogBlockInputStream::readImpl() while (true) { + Stopwatch watch; size_t new_rows = 0; - exception_message.reset(); if (buffer->poll()) { try @@ -122,28 +114,16 @@ Block FileLogBlockInputStream::readImpl() total_rows = total_rows + new_rows; } - if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) + if ((!buffer->hasMorePolledRecords() && (total_rows >= max_block_size)) || watch.elapsedMilliseconds() > poll_time_out) { break; } } if (total_rows == 0) - return Block(); + return {}; - auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); - - return ConvertingBlockInputStream( - std::make_shared(result_block), - getHeader(), - ConvertingBlockInputStream::MatchColumnsMode::Name) - .read(); -} - -void FileLogBlockInputStream::readSuffixImpl() -{ - if (buffer) - buffer->close(); + return Chunk(std::move(result_columns), total_rows); } } diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogSource.h similarity index 57% rename from src/Storages/FileLog/FileLogBlockInputStream.h rename to src/Storages/FileLog/FileLogSource.h index d72ab5649ee..042a973986c 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -12,26 +12,24 @@ namespace Poco } namespace DB { -class FileLogBlockInputStream : public IBlockInputStream +class FileLogSource : public SourceWithProgress { public: - FileLogBlockInputStream( + FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_, + const ContextPtr & context_, const Names & columns, - size_t max_block_size_); - ~FileLogBlockInputStream() override = default; - - String getName() const override { return storage.getName(); } - Block getHeader() const override; - - void readPrefixImpl() override; - Block readImpl() override; - void readSuffixImpl() override; + size_t max_block_size_, + size_t poll_time_out_, + size_t stream_number_, + size_t max_streams_number_); bool isStalled() { return !buffer || buffer->isStalled(); } +protected: + Chunk generate() override; + private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; @@ -39,10 +37,19 @@ private: Names column_names; UInt64 max_block_size; + size_t poll_time_out; + + size_t stream_number; + size_t max_streams_number; + ReadBufferFromFileLogPtr buffer; + bool started = false; + const Block non_virtual_header; const Block virtual_header; + + void createReadBuffer(); }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index e5849b50894..1c39296dabb 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -1,13 +1,11 @@ #include #include -#include -#include #include -#include -#include #include +#include +#include namespace DB { @@ -29,33 +27,9 @@ ReadBufferFromFileLog::ReadBufferFromFileLog( void ReadBufferFromFileLog::open() { - Poco::File file(path); - - if (file.isFile()) - { - file_status[path].reader = std::ifstream(path); - } - else if (file.isDirectory()) - { - path_is_directory = true; - Poco::DirectoryIterator dir_iter(file); - Poco::DirectoryIterator end; - while (dir_iter != end) - { - if (dir_iter->isFile()) - file_status[dir_iter->path()].reader = std::ifstream(dir_iter->path()); - ++dir_iter; - } - } - wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); wait_task->deactivate(); - if (path_is_directory) - { - select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); - select_task->activateAndSchedule(); - } cleanUnprocessed(); allowed = false; @@ -183,43 +157,4 @@ void ReadBufferFromFileLog::waitFunc() time_out = true; } -void ReadBufferFromFileLog::watchFunc() -{ - FileLogDirectoryWatcher dw(path); - while (true) - { - sleepForNanoseconds(poll_timeout); - - auto error = dw.getError(); - if (error) - LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); - - auto events = dw.getEvents(); - std::lock_guard lock(status_mutex); - - for (const auto & event : events) - { - switch (event.type) - { - - case Poco::DirectoryWatcher::DW_ITEM_ADDED: - LOG_TRACE(log, "New event {} watched.", event.callback); - file_status[event.path].reader = std::ifstream(event.path); - break; - - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched.", event.callback); - file_status[event.path].status = FileStatus::UPDATED; - break; - - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched.", event.callback); - file_status[event.path].status = FileStatus::REMOVED; - break; - } - } - } -} } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index e74c93d189e..e7385282f71 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -33,26 +33,13 @@ public: bool poll(); - bool isStalled() { return buffer_status != BufferStatus::NOT_STALLED; } + bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } private: enum class BufferStatus { NO_RECORD_RETURNED, - NOT_STALLED, - }; - enum class FileStatus - { - BEGIN, - NO_CHANGE, - UPDATED, - REMOVED - }; - - struct FileContext - { - FileStatus status = FileStatus::BEGIN; - std::ifstream reader; + POLLED_OK, }; BufferStatus buffer_status; @@ -67,10 +54,6 @@ private: bool time_out = false; - using NameToFile = std::unordered_map; - NameToFile file_status; - - std::mutex status_mutex; ContextPtr context; @@ -85,7 +68,6 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; TaskThread wait_task; - TaskThread select_task; Records pollBatch(size_t batch_size_); @@ -97,6 +79,5 @@ private: void waitFunc(); - [[noreturn]] void watchFunc(); }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 565b206e2cc..a6f819580d5 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,16 +1,3 @@ -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -18,20 +5,25 @@ #include #include #include +#include #include #include +#include +#include +#include #include #include -#include -#include #include #include #include +#include #include #include #include #include +#include +#include namespace DB { @@ -53,19 +45,41 @@ StorageFileLog::StorageFileLog( ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & format_name_) + const String & format_name_, + std::unique_ptr settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , filelog_settings(std::move(settings)) , path(path_) , format_name(format_name_) - , log(&Poco::Logger::get("StorageFile (" + table_id_.table_name + ")")) + , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + if (std::filesystem::is_regular_file(path)) + { + file_status[path].reader = std::ifstream(path); + file_names.push_back(path); + } + else if (std::filesystem::is_directory(path)) + { + path_is_directory = true; + /// Just consider file with depth 1 + for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + { + if (dir_entry.is_regular_file()) + { + file_status[dir_entry.path()].reader = std::ifstream(dir_entry.path()); + file_names.push_back(dir_entry.path()); + } + } + } + + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - thread->deactivate(); task = std::make_shared(std::move(thread)); } @@ -78,23 +92,38 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { + std::lock_guard lock(status_mutex); auto modified_context = Context::createCopy(local_context); - return Pipe(std::make_shared( - std::make_shared(*this, metadata_snapshot, modified_context, column_names, 1))); + clearInvalidFiles(); + + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + /// No files to parse + if (max_streams_number == 0) + { + return Pipe{}; + } + + Pipes pipes; + pipes.reserve(max_streams_number); + for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) + { + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + modified_context, + column_names, + getMaxBlockSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); + } + + return Pipe::unitePipes(std::move(pipes)); } void StorageFileLog::startup() { - try - { - createReadBuffer(); - } - catch (const Exception &) - { - tryLogCurrentException(log); - } - task->holder->activateAndSchedule(); } @@ -105,26 +134,32 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); + watch_task->deactivate(); - LOG_TRACE(log, "Closing files"); - destroyReadBuffer(); + for (auto & file : file_status) + { + file.second.reader.close(); + } } size_t StorageFileLog::getMaxBlockSize() const { - return getContext()->getSettingsRef().max_insert_block_size.value; + return filelog_settings->filelog_max_block_size.changed ? filelog_settings->filelog_max_block_size.value + : getContext()->getSettingsRef().max_insert_block_size.value; } size_t StorageFileLog::getPollMaxBatchSize() const { - size_t batch_size = getContext()->getSettingsRef().max_block_size.value; + size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value + : getContext()->getSettingsRef().max_block_size.value; - return std::min(batch_size,getMaxBlockSize()); + return std::min(batch_size, getMaxBlockSize()); } size_t StorageFileLog::getPollTimeoutMillisecond() const { - return getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); + return filelog_settings->filelog_poll_timeout_ms.changed ? filelog_settings->filelog_poll_timeout_ms.totalMilliseconds() + : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } @@ -135,9 +170,9 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) if (dependencies.empty()) return true; - for (const auto & db_tab : dependencies) + for (const auto & storage : dependencies) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); + auto table = DatabaseCatalog::instance().tryGetTable(storage, getContext()); if (!table) return false; @@ -147,25 +182,13 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) return false; // Check all its dependencies - if (!checkDependencies(db_tab)) + if (!checkDependencies(storage)) return false; } return true; } -void StorageFileLog::createReadBuffer() -{ - auto new_context = Context::createCopy(getContext()); - buffer = std::make_shared(path, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); -} - -void StorageFileLog::destroyReadBuffer() -{ - if (buffer) - buffer->close(); -} - void StorageFileLog::threadFunc() { try @@ -177,6 +200,8 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); + watch_task->activateAndSchedule(); + // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) { @@ -207,6 +232,7 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } + watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -215,6 +241,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { + std::lock_guard lock(status_mutex); Stopwatch watch; auto table_id = getStorageID(); @@ -223,41 +250,79 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); + clearInvalidFiles(); + + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + /// No files to parse + if (max_streams_number == 0) + { + return false; + } + // Create an INSERT query for streaming data auto insert = std::make_shared(); insert->table_id = table_id; - size_t block_size = getMaxBlockSize(); - auto new_context = Context::createCopy(getContext()); InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); - auto stream = std::make_shared( - *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), block_size); + Pipes pipes; + pipes.reserve(max_streams_number); + for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) + { + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + new_context, + block_io.out->getHeader().getNames(), + getPollMaxBatchSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); + } - StreamLocalLimits limits; + QueryPipeline pipeline; + pipeline.init(Pipe::unitePipes(std::move(pipes))); - limits.speed_limits.max_execution_time = getContext()->getSettingsRef().stream_flush_interval_ms; - - limits.timeout_overflow_mode = OverflowMode::BREAK; - stream->setLimits(limits); - - std::atomic stub = {false}; size_t rows = 0; - copyData( - *stream, *block_io.out, [&rows](const Block & block) { rows += block.rows(); }, &stub); - bool stream_is_stalled = false; - - stream_is_stalled = stream->as()->isStalled(); + PullingPipelineExecutor executor(pipeline); + Block block; + block_io.out->writePrefix(); + while (executor.pull(block)) + { + block_io.out->write(block); + rows += block.rows(); + } + block_io.out->writeSuffix(); UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return stream_is_stalled; + return true; +} + +void StorageFileLog::clearInvalidFiles() +{ + /// Do not need to hold file_status lock, since it will be holded + /// by caller when call this function + std::vector valid_files; + for (const auto & it : file_names) + { + if (file_status.at(it).status == FileStatus::REMOVED) + { + file_status.erase(it); + } + else + { + valid_files.push_back(it); + } + } + + file_names.swap(valid_files); } void registerStorageFileLog(StorageFactory & factory) @@ -267,6 +332,36 @@ void registerStorageFileLog(StorageFactory & factory) ASTs & engine_args = args.engine_args; size_t args_count = engine_args.size(); + bool has_settings = args.storage_def->settings; + + auto filelog_settings = std::make_unique(); + if (has_settings) + { + filelog_settings->loadFromQuery(*args.storage_def); + } + + auto physical_cpu_cores = getNumberOfPhysicalCPUCores(); + auto num_threads = filelog_settings->filelog_max_threads.value; + + if (num_threads > physical_cpu_cores) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of threads to parse files can not be bigger than {}", physical_cpu_cores); + } + else if (num_threads < 1) + { + throw Exception("Number of threads to parse files can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); + } + + if (filelog_settings->filelog_max_block_size.changed && filelog_settings->filelog_max_block_size.value < 1) + { + throw Exception("filelog_max_block_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); + } + + if (filelog_settings->filelog_poll_max_batch_size.changed && filelog_settings->filelog_poll_max_batch_size.value < 1) + { + throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); + } + if (args_count != 2) throw Exception( "Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -277,7 +372,7 @@ void registerStorageFileLog(StorageFactory & factory) auto path = path_ast->as().value.safeGet(); auto format = format_ast->as().value.safeGet(); - return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format); + return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings)); }; factory.registerStorage( @@ -300,4 +395,54 @@ Names StorageFileLog::getVirtualColumnNames() return result; } +void StorageFileLog::watchFunc() +{ + FileLogDirectoryWatcher dw(path); + while (true) + { + sleepForMicroseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); + + auto error = dw.getError(); + if (error) + LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); + + auto events = dw.getEvents(); + std::lock_guard lock(status_mutex); + + for (const auto & event : events) + { + switch (event.type) + { + + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path)) + { + file_status[event.path].reader = std::ifstream(event.path); + file_names.push_back(event.path); + } + break; + + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::UPDATED; + } + break; + + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::REMOVED; + } + break; + } + } + } +} + } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 93b67fe5a31..314724d73be 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -11,8 +12,8 @@ #include #include -#include #include +#include namespace DB { @@ -52,16 +53,39 @@ protected: ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & format_name_); + const String & format_name_, + std::unique_ptr settings); private: + std::unique_ptr filelog_settings; const String path; + bool path_is_directory = false; + const String format_name; Poco::Logger * log; ReadBufferFromFileLogPtr buffer; - std::mutex mutex; + enum class FileStatus + { + BEGIN, + NO_CHANGE, + UPDATED, + REMOVED + }; + + struct FileContext + { + FileStatus status = FileStatus::BEGIN; + std::ifstream reader; + }; + + using NameToFile = std::unordered_map; + NameToFile file_status; + + std::vector file_names; + + std::mutex status_mutex; // Stream thread struct TaskContext @@ -74,17 +98,22 @@ private: }; std::shared_ptr task; - void createReadBuffer(); - void destroyReadBuffer(); + using TaskThread = BackgroundSchedulePool::TaskHolder; + + TaskThread watch_task; void threadFunc(); + void clearInvalidFiles(); + size_t getPollMaxBatchSize() const; size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; bool streamToViews(); bool checkDependencies(const StorageID & table_id); + + [[noreturn]] void watchFunc(); }; } From 09bc3d723af90e2ae412e1728b775ee0a99a0333 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 06:32:32 +0000 Subject: [PATCH 011/950] refactor some code --- src/Storages/FileLog/FileLogSource.cpp | 9 +-- src/Storages/FileLog/FileLogSource.h | 11 +-- .../FileLog/ReadBufferFromFileLog.cpp | 73 ++++++++----------- src/Storages/FileLog/ReadBufferFromFileLog.h | 32 ++++---- src/Storages/FileLog/StorageFileLog.cpp | 10 +-- src/Storages/FileLog/StorageFileLog.h | 22 +++--- 6 files changed, 64 insertions(+), 93 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 4a15cd07f25..4827a048d6e 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -24,19 +24,18 @@ FileLogSource::FileLogSource( size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : storage(storage_) + : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) + , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) - , stream_number(stream_number_) - , max_streams_number(max_streams_number_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { - createReadBuffer(); + buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } Chunk FileLogSource::generate() @@ -114,7 +113,7 @@ Chunk FileLogSource::generate() total_rows = total_rows + new_rows; } - if ((!buffer->hasMorePolledRecords() && (total_rows >= max_block_size)) || watch.elapsedMilliseconds() > poll_time_out) + if (!buffer->hasMorePolledRecords() && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out)) { break; } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 042a973986c..7db2602dc8c 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -25,7 +25,9 @@ public: size_t stream_number_, size_t max_streams_number_); - bool isStalled() { return !buffer || buffer->isStalled(); } + String getName() const override { return "FileLog"; } + + bool noRecords() { return !buffer || buffer->noRecords(); } protected: Chunk generate() override; @@ -39,17 +41,12 @@ private: size_t poll_time_out; - size_t stream_number; - size_t max_streams_number; - - ReadBufferFromFileLogPtr buffer; + std::unique_ptr buffer; bool started = false; const Block non_virtual_header; const Block virtual_header; - - void createReadBuffer(); }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 1c39296dabb..4df13199901 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -15,22 +16,21 @@ namespace ErrorCodes } ReadBufferFromFileLog::ReadBufferFromFileLog( - const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_) + StorageFileLog & storage_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_, + size_t stream_number_, + size_t max_streams_number_) : ReadBuffer(nullptr, 0) - , path(path_) - , log(log_) + , log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number))) + , storage(storage_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , context(context_) + , stream_number(stream_number_) + , max_streams_number(max_streams_number_) { -} - -void ReadBufferFromFileLog::open() -{ - wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); - wait_task->deactivate(); - - cleanUnprocessed(); allowed = false; } @@ -42,17 +42,6 @@ void ReadBufferFromFileLog::cleanUnprocessed() BufferBase::set(nullptr, 0, 0); } -void ReadBufferFromFileLog::close() -{ - wait_task->deactivate(); - - if (path_is_directory) - select_task->deactivate(); - - for (auto & status : file_status) - status.second.reader.close(); -} - bool ReadBufferFromFileLog::poll() { @@ -77,7 +66,7 @@ bool ReadBufferFromFileLog::poll() LOG_TRACE(log, "Polled batch of {} records. ", records.size()); - buffer_status = BufferStatus::NOT_STALLED; + buffer_status = BufferStatus::POLLED_OK; allowed = true; return true; } @@ -94,43 +83,45 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz if (new_records.size() == batch_size_) return new_records; - wait_task->activateAndSchedule(); - while (!time_out && new_records.size() != batch_size_) + Stopwatch watch; + while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) { readNewRecords(new_records, batch_size); } - wait_task->deactivate(); - time_out = false; return new_records; } +// TODO void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) { - std::lock_guard lock(status_mutex); - size_t need_records_size = batch_size_ - new_records.size(); size_t read_records_size = 0; - for (auto & status : file_status) + const auto & file_names = storage.getFileNames(); + auto & file_status = storage.getFileStatus(); + + size_t files_per_stream = file_names.size() / max_streams_number; + size_t start = stream_number * files_per_stream; + size_t end = stream_number == max_streams_number - 1 ? file_names.size() : (stream_number + 1) * files_per_stream; + + for (size_t i = start; i < end; ++i) { - if (status.second.status == FileStatus::NO_CHANGE) + auto & file = file_status[file_names[i]]; + if (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::REMOVED) continue; - if (status.second.status == FileStatus::REMOVED) - file_status.erase(status.first); - - while (read_records_size < need_records_size && status.second.reader.good() && !status.second.reader.eof()) + while (read_records_size < need_records_size && file.reader.good() && !file.reader.eof()) { Record record; - std::getline(status.second.reader, record); + std::getline(file.reader, record); new_records.emplace_back(record); ++read_records_size; } // Read to the end of the file - if (status.second.reader.eof()) - status.second.status = FileStatus::NO_CHANGE; + if (file.reader.eof()) + file.status = StorageFileLog::FileStatus::NO_CHANGE; if (read_records_size == need_records_size) break; @@ -151,10 +142,4 @@ bool ReadBufferFromFileLog::nextImpl() return true; } -void ReadBufferFromFileLog::waitFunc() -{ - sleepForMicroseconds(poll_timeout); - time_out = true; -} - } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index e7385282f71..dc5f4028b8d 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -3,12 +3,11 @@ #include #include #include -#include +#include #include #include #include -#include namespace Poco { @@ -20,16 +19,19 @@ namespace DB class ReadBufferFromFileLog : public ReadBuffer { public: - ReadBufferFromFileLog(const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_); + ReadBufferFromFileLog( + StorageFileLog & storage_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_, + size_t stream_number_, + size_t max_streams_number_); ~ReadBufferFromFileLog() override = default; - void open(); - void close(); - auto pollTimeout() const { return poll_timeout; } - inline bool hasMorePolledRecords() const { return current != records.end(); } + bool hasMorePolledRecords() const { return current != records.end(); } bool poll(); @@ -44,19 +46,18 @@ private: BufferStatus buffer_status; - const String path; - - bool path_is_directory = false; - Poco::Logger * log; + + StorageFileLog & storage; + const size_t batch_size = 1; const size_t poll_timeout = 0; - bool time_out = false; - - ContextPtr context; + size_t stream_number; + size_t max_streams_number; + bool allowed = true; using Record = std::string; @@ -76,8 +77,5 @@ private: void cleanUnprocessed(); bool nextImpl() override; - - void waitFunc(); - }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a6f819580d5..5df8a416701 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -383,16 +384,9 @@ void registerStorageFileLog(StorageFactory & factory) }); } -NamesAndTypesList StorageFileLog::getVirtuals() const -{ - auto result = NamesAndTypesList{}; - return result; -} - Names StorageFileLog::getVirtualColumnNames() { - auto result = Names{}; - return result; + return {}; } void StorageFileLog::watchFunc() diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 314724d73be..3e5edea33ef 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -22,6 +22,14 @@ class StorageFileLog final : public shared_ptr_helper, public IS friend struct shared_ptr_helper; public: + enum class FileStatus + { + BEGIN, + NO_CHANGE, + UPDATED, + REMOVED + }; + using Files = std::vector; std::string getName() const override { return "FileLog"; } @@ -42,10 +50,10 @@ public: const auto & getFormatName() const { return format_name; } - NamesAndTypesList getVirtuals() const override; static Names getVirtualColumnNames(); - auto & getBuffer() { return buffer; } + auto & getFileNames() { return file_names; } + auto & getFileStatus() { return file_status; } protected: StorageFileLog( @@ -64,16 +72,6 @@ private: const String format_name; Poco::Logger * log; - ReadBufferFromFileLogPtr buffer; - - enum class FileStatus - { - BEGIN, - NO_CHANGE, - UPDATED, - REMOVED - }; - struct FileContext { FileStatus status = FileStatus::BEGIN; From dfea640c67ae88e7ab4785993f717b1517449816 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 11:41:13 +0000 Subject: [PATCH 012/950] fix --- .../FileLog/FileLogDirectoryWatcher.cpp | 2 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/FileLogSource.cpp | 24 ++++++++++++------- src/Storages/FileLog/FileLogSource.h | 6 ++--- .../FileLog/ReadBufferFromFileLog.cpp | 1 - src/Storages/FileLog/ReadBufferFromFileLog.h | 4 ++-- src/Storages/FileLog/StorageFileLog.cpp | 14 +++++------ src/Storages/FileLog/StorageFileLog.h | 2 -- 8 files changed, 29 insertions(+), 26 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 931f9c9aa06..bc731500d61 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -20,7 +20,7 @@ FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() return res; } -bool FileLogDirectoryWatcher::getError() const +bool FileLogDirectoryWatcher::hasError() const { return error; } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 99fd5cf0af1..651d1d9db2d 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -24,7 +24,7 @@ public: Events getEvents(); - bool getError() const; + bool hasError() const; const std::string & getPath() const; diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 4827a048d6e..ecc43c579c7 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -24,16 +24,15 @@ FileLogSource::FileLogSource( size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) - , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) - , virtual_header( - metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) + , non_virtual_header(metadata_snapshot_->getSampleBlockNonMaterialized()) + , column_names_and_types(metadata_snapshot_->getColumns().getByNames(ColumnsDescription::All, columns, true)) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } @@ -43,7 +42,7 @@ Chunk FileLogSource::generate() if (!buffer) return {}; - MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); + MutableColumns read_columns = non_virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); @@ -79,7 +78,7 @@ Chunk FileLogSource::generate() auto columns = chunk.detachColumns(); for (size_t i = 0, s = columns.size(); i < s; ++i) { - result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + read_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); } break; } @@ -103,9 +102,9 @@ Chunk FileLogSource::generate() { new_rows = read_file_log(); } - catch (Exception &) + catch (...) { - throw; + tryLogCurrentException(__PRETTY_FUNCTION__); } } if (new_rows) @@ -122,6 +121,15 @@ Chunk FileLogSource::generate() if (total_rows == 0) return {}; + Columns result_columns; + result_columns.reserve(column_names_and_types.size()); + + for (const auto & elem : column_names_and_types) + { + auto index = non_virtual_header.getPositionByName(elem.getNameInStorage()); + result_columns.emplace_back(std::move(read_columns[index])); + } + return Chunk(std::move(result_columns), total_rows); } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 7db2602dc8c..1dc55f2c0c2 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -43,10 +43,8 @@ private: std::unique_ptr buffer; - bool started = false; - - const Block non_virtual_header; - const Block virtual_header; + Block non_virtual_header; + const NamesAndTypesList column_names_and_types; }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 4df13199901..0c164c6b070 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -92,7 +92,6 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz return new_records; } -// TODO void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) { size_t need_records_size = batch_size_ - new_records.size(); diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index dc5f4028b8d..a7447e5edcd 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -50,8 +50,8 @@ private: StorageFileLog & storage; - const size_t batch_size = 1; - const size_t poll_timeout = 0; + size_t batch_size; + size_t poll_timeout; ContextPtr context; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 5df8a416701..cb083baddb2 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -77,6 +77,10 @@ StorageFileLog::StorageFileLog( } } } + else + { + throw Exception("The path neigher a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + } watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); @@ -384,23 +388,19 @@ void registerStorageFileLog(StorageFactory & factory) }); } -Names StorageFileLog::getVirtualColumnNames() -{ - return {}; -} - void StorageFileLog::watchFunc() { FileLogDirectoryWatcher dw(path); while (true) { - sleepForMicroseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); + sleepForMilliseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); - auto error = dw.getError(); + auto error = dw.hasError(); if (error) LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); auto events = dw.getEvents(); + std::lock_guard lock(status_mutex); for (const auto & event : events) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 3e5edea33ef..ada9ebf4004 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -50,8 +50,6 @@ public: const auto & getFormatName() const { return format_name; } - static Names getVirtualColumnNames(); - auto & getFileNames() { return file_names; } auto & getFileStatus() { return file_status; } From b97faf8b25d9464468b9394342330606ed45c7f3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 12:12:04 +0000 Subject: [PATCH 013/950] fix style --- src/Storages/FileLog/StorageFileLog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index cb083baddb2..7d14df0379f 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; } namespace @@ -79,7 +80,7 @@ StorageFileLog::StorageFileLog( } else { - throw Exception("The path neigher a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); } watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); From 580b047d9699ea1633611f867b3e1e1feae6478f Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 13:45:08 +0000 Subject: [PATCH 014/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7d14df0379f..63a282b3a20 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -98,7 +98,7 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); auto modified_context = Context::createCopy(local_context); clearInvalidFiles(); @@ -206,7 +206,7 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activateAndSchedule(); + watch_task->activateAndSchedule(); // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) @@ -238,7 +238,7 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - watch_task->deactivate(); + watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -247,7 +247,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); Stopwatch watch; auto table_id = getStorageID(); @@ -292,6 +292,8 @@ bool StorageFileLog::streamToViews() QueryPipeline pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); + assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); + size_t rows = 0; PullingPipelineExecutor executor(pipeline); @@ -394,7 +396,7 @@ void StorageFileLog::watchFunc() FileLogDirectoryWatcher dw(path); while (true) { - sleepForMilliseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); + sleepForMilliseconds(getPollTimeoutMillisecond()); auto error = dw.hasError(); if (error) @@ -402,10 +404,9 @@ void StorageFileLog::watchFunc() auto events = dw.getEvents(); - std::lock_guard lock(status_mutex); - for (const auto & event : events) { + std::lock_guard lock(status_mutex); switch (event.type) { From a22a0d6c68459763c2e0fd6a3a63f84aa64cfdfa Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 13:47:20 +0000 Subject: [PATCH 015/950] fix style --- src/Storages/FileLog/StorageFileLog.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 63a282b3a20..ba64a590e63 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -98,7 +98,8 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); + auto modified_context = Context::createCopy(local_context); clearInvalidFiles(); @@ -206,7 +207,7 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activateAndSchedule(); + watch_task->activateAndSchedule(); // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) @@ -238,7 +239,7 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - watch_task->deactivate(); + watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -247,7 +248,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); Stopwatch watch; auto table_id = getStorageID(); @@ -406,7 +407,7 @@ void StorageFileLog::watchFunc() for (const auto & event : events) { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); switch (event.type) { From 3ce733121131da78380bab99db545dff00abf21b Mon Sep 17 00:00:00 2001 From: George Date: Tue, 7 Sep 2021 20:07:27 +0300 Subject: [PATCH 016/950] Edited and translated to Russian --- .../table-engines/integrations/mongodb.md | 4 ++-- .../external-dicts-dict-layout.md | 5 +++-- .../table-engines/integrations/mongodb.md | 20 ++++++++++++++++--- docs/ru/operations/settings/settings-users.md | 2 +- .../external-dicts-dict-layout.md | 20 ++++++++++++++++--- .../sql-reference/statements/create/user.md | 1 + 6 files changed, 41 insertions(+), 11 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/mongodb.md b/docs/en/engines/table-engines/integrations/mongodb.md index 9839893d4e8..899db9ac6ae 100644 --- a/docs/en/engines/table-engines/integrations/mongodb.md +++ b/docs/en/engines/table-engines/integrations/mongodb.md @@ -36,7 +36,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name Create a table in ClickHouse which allows to read data from MongoDB collection: -``` text +``` sql CREATE TABLE mongo_table ( key UInt64, @@ -46,7 +46,7 @@ CREATE TABLE mongo_table To read from an SSL secured MongoDB server: -``` text +``` sql CREATE TABLE mongo_table_ssl ( key UInt64, diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 00a9610ce91..eb8ca425d24 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -300,8 +300,9 @@ When searching for a dictionary, the cache is searched first. For each block of If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. -For cache dictionaries, the expiration [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired, and it is re-requested the next time it needs to be used this behaviour can be configured with setting `allow_read_expired_keys`. -This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the `system.dictionaries` table. +For cache dictionaries, the expiration [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. + +This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. If setting `allow_read_expired_keys` is set to 1, by default 0. Then dictionary can support asynchronous updates. If a client requests keys and all of them are in cache, but some of them are expired, then dictionary will return expired keys for a client and request them asynchronously from the source. diff --git a/docs/ru/engines/table-engines/integrations/mongodb.md b/docs/ru/engines/table-engines/integrations/mongodb.md index 05820d03fe6..0fd463d511a 100644 --- a/docs/ru/engines/table-engines/integrations/mongodb.md +++ b/docs/ru/engines/table-engines/integrations/mongodb.md @@ -15,7 +15,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name name1 [type1], name2 [type2], ... -) ENGINE = MongoDB(host:port, database, collection, user, password); +) ENGINE = MongoDB(host:port, database, collection, user, password [, options]); ``` **Параметры движка** @@ -30,11 +30,13 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name - `password` — пароль пользователя. +- `options` — MongoDB connection string options (optional parameter). + ## Примеры использования {#usage-example} -Таблица в ClickHouse для чтения данных из колекции MongoDB: +Создание таблицы в ClickHouse для чтения данных из колекции MongoDB: -``` text +``` sql CREATE TABLE mongo_table ( key UInt64, @@ -42,6 +44,18 @@ CREATE TABLE mongo_table ) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'testuser', 'clickhouse'); ``` +Чтение из сервера MongoDB, защищенного SSL: + +``` sql +CREATE TABLE mongo_table_ssl +( + key UInt64, + data String +) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table', 'testuser', 'clickhouse', 'ssl=true'); +``` + + + Запрос к таблице: ``` sql diff --git a/docs/ru/operations/settings/settings-users.md b/docs/ru/operations/settings/settings-users.md index 6a10e518817..0f85f22f7ea 100644 --- a/docs/ru/operations/settings/settings-users.md +++ b/docs/ru/operations/settings/settings-users.md @@ -28,7 +28,7 @@ toc_title: "Настройки пользователей" profile_name default - + default_database>default diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 06fe4ae327a..b61c2cbcbd7 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -297,9 +297,13 @@ RANGE(MIN StartDate MAX EndDate); При поиске в словаре сначала просматривается кэш. На каждый блок данных, все не найденные в кэше или устаревшие ключи запрашиваются у источника с помощью `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. Затем, полученные данные записываются в кэш. -Для cache-словарей может быть задано время устаревания [lifetime](external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, и будет запрошено заново при следующей необходимости его использовать. +Если ключи не были найдены в словаре, то создается задание для обновления кэша, которое добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` -Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа cache показывает высокую производительность лишь при достаточно больших hit rate-ах (рекомендуется 99% и выше). Посмотреть средний hit rate можно в таблице `system.dictionaries`. +Для cache-словарей может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это можно настроить с помощью `allow_read_expired_keys`. + +Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа cache показывает высокую производительность лишь при достаточно больших hit rate-ах (рекомендуется 99% и выше). Посмотреть средний hit rate можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). + +Если параметр `allow_read_expired_keys` выставлен на 1 (0 по умолчанию), то словарь поддерживает асинхронные обновления. Если клиент запрашивает ключи, которые находятся в кэше, но при этом некоторые из них устарели, то словарь вернет устаревшие ключи клиенту и запросит их асинхронно у источника. Чтобы увеличить производительность кэша, используйте подзапрос с `LIMIT`, а снаружи вызывайте функцию со словарём. @@ -312,6 +316,16 @@ RANGE(MIN StartDate MAX EndDate); 1000000000 + + 0 + + 100000 + + 10 + + 60000 + + 4 ``` @@ -338,7 +352,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) ### ssd_cache {#ssd-cache} -Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. +Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. Все параметры, относящиеся к очереди обновлений, могут также быть применены к SSD-кэш словарям. ``` xml diff --git a/docs/ru/sql-reference/statements/create/user.md b/docs/ru/sql-reference/statements/create/user.md index 22efaa71bfc..f6248d97ba9 100644 --- a/docs/ru/sql-reference/statements/create/user.md +++ b/docs/ru/sql-reference/statements/create/user.md @@ -15,6 +15,7 @@ CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [DEFAULT ROLE role [,...]] + [DEFAULT DATABASE database | NONE] [GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY | WRITABLE] | PROFILE 'profile_name'] [,...] ``` From 8f328ca6c4bb07c5918e9ad0059101d5a715638e Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Fri, 10 Sep 2021 03:26:10 +0300 Subject: [PATCH 017/950] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/table-engines/integrations/mongodb.md | 4 ++-- docs/ru/operations/settings/settings-users.md | 2 +- .../external-dictionaries/external-dicts-dict-layout.md | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/mongodb.md b/docs/ru/engines/table-engines/integrations/mongodb.md index 0fd463d511a..7a56af4c274 100644 --- a/docs/ru/engines/table-engines/integrations/mongodb.md +++ b/docs/ru/engines/table-engines/integrations/mongodb.md @@ -34,7 +34,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ## Примеры использования {#usage-example} -Создание таблицы в ClickHouse для чтения данных из колекции MongoDB: +Создание таблицы в ClickHouse для чтения данных из коллекции MongoDB: ``` sql CREATE TABLE mongo_table @@ -44,7 +44,7 @@ CREATE TABLE mongo_table ) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'testuser', 'clickhouse'); ``` -Чтение из сервера MongoDB, защищенного SSL: +Чтение с сервера MongoDB, защищенного SSL: ``` sql CREATE TABLE mongo_table_ssl diff --git a/docs/ru/operations/settings/settings-users.md b/docs/ru/operations/settings/settings-users.md index 0f85f22f7ea..4570ce38bad 100644 --- a/docs/ru/operations/settings/settings-users.md +++ b/docs/ru/operations/settings/settings-users.md @@ -28,7 +28,7 @@ toc_title: "Настройки пользователей" profile_name default - default_database>default + default diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index b61c2cbcbd7..f0b4eb614c5 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -297,13 +297,13 @@ RANGE(MIN StartDate MAX EndDate); При поиске в словаре сначала просматривается кэш. На каждый блок данных, все не найденные в кэше или устаревшие ключи запрашиваются у источника с помощью `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. Затем, полученные данные записываются в кэш. -Если ключи не были найдены в словаре, то создается задание для обновления кэша, которое добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` +Если ключи не были найдены в словаре, то для обновления кэша создается задание и добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` Для cache-словарей может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это можно настроить с помощью `allow_read_expired_keys`. -Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа cache показывает высокую производительность лишь при достаточно больших hit rate-ах (рекомендуется 99% и выше). Посмотреть средний hit rate можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). +Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа `cache` показывает высокую производительность лишь при достаточно большой частоте успешных обращений (рекомендуется 99% и выше). Посмотреть среднюю частоту успешных обращений (`hit rate`) можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). -Если параметр `allow_read_expired_keys` выставлен на 1 (0 по умолчанию), то словарь поддерживает асинхронные обновления. Если клиент запрашивает ключи, которые находятся в кэше, но при этом некоторые из них устарели, то словарь вернет устаревшие ключи клиенту и запросит их асинхронно у источника. +Если параметр `allow_read_expired_keys` выставлен в 1 (0 по умолчанию), то словарь поддерживает асинхронные обновления. Если клиент запрашивает ключи, которые находятся в кэше, но при этом некоторые из них устарели, то словарь вернет устаревшие ключи клиенту и запросит их асинхронно у источника. Чтобы увеличить производительность кэша, используйте подзапрос с `LIMIT`, а снаружи вызывайте функцию со словарём. @@ -352,7 +352,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) ### ssd_cache {#ssd-cache} -Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. Все параметры, относящиеся к очереди обновлений, могут также быть применены к SSD-кэш словарям. +Похож на `cache`, но хранит данные на SSD, а индекс в оперативной памяти. Все параметры, относящиеся к очереди обновлений, могут также быть применены к SSD-кэш словарям. ``` xml From ae5ee23c83e75035653f2571540474ee5e661f07 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 12:12:11 +0300 Subject: [PATCH 018/950] Trying self hosted action --- .github/workflows/hello-world.yml | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 .github/workflows/hello-world.yml diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml new file mode 100644 index 00000000000..5dd0429bce7 --- /dev/null +++ b/.github/workflows/hello-world.yml @@ -0,0 +1,23 @@ +name: GitHub Actions Hello self hosted +on: + push: + branches: + - master + pull_request: + branches: + - master +jobs: + Explore-GitHub-Actions: + runs-on: [self-hosted] + steps: + - run: echo "🎉 The job was automatically triggered by a ${{ github.event_name }} event." + - run: echo "🐧 This job is now running on a ${{ runner.os }} server hosted by me!" + - run: echo "🔎 The name of your branch is ${{ github.ref }} and your repository is ${{ github.repository }}." + - name: Check out repository code + uses: actions/checkout@v2 + - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." + - run: echo "🖥️ The workflow is now ready to test your code on the runner." + - name: List files in the repository + run: | + ls ${{ github.workspace }} + - run: echo "🍏 This job's status is ${{ job.status }}." From 9dc7e00c2e2bb69b1429b18cbb27c05dcb6c3561 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 12:33:41 +0300 Subject: [PATCH 019/950] Trying docker --- .github/workflows/hello-world.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 5dd0429bce7..ed0cf36547e 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -13,6 +13,7 @@ jobs: - run: echo "🎉 The job was automatically triggered by a ${{ github.event_name }} event." - run: echo "🐧 This job is now running on a ${{ runner.os }} server hosted by me!" - run: echo "🔎 The name of your branch is ${{ github.ref }} and your repository is ${{ github.repository }}." + - run: docker run hello-world - name: Check out repository code uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." From 4014b5c11177c44cc6f4c85e7d9edf447ee4deb1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 12:55:31 +0300 Subject: [PATCH 020/950] Test --- .github/workflows/hello-world.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index ed0cf36547e..8ba33da6d5d 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -18,6 +18,7 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." + - run: docker run --cap-add=SYS_PTRACE --volume=${{ github.workspace }}:/ClickHouse --volume=${{ github.workspace }}:/test_output clickhouse/style-test:latest - name: List files in the repository run: | ls ${{ github.workspace }} From b4107784f14552d7e26e5fab05e3c85c6ea7de65 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 13:01:02 +0300 Subject: [PATCH 021/950] Better --- .github/workflows/hello-world.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 8ba33da6d5d..9ef1c19fd3a 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -18,8 +18,10 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." - - run: docker run --cap-add=SYS_PTRACE --volume=${{ github.workspace }}:/ClickHouse --volume=${{ github.workspace }}:/test_output clickhouse/style-test:latest + - run: mkdir -p $RUNNER_TEMP/style_check_result + - run: docker run --cap-add=SYS_PTRACE --volume=$GITHUB_WORKSPACE:/ClickHouse --volume=$RUNNER_TEMP/style_check_result:/test_output clickhouse/style-test:latest - name: List files in the repository run: | ls ${{ github.workspace }} + ls $RUNNER_TEMP/style_check_result - run: echo "🍏 This job's status is ${{ job.status }}." From b6219376e334b3049ecb802a9b37ff4c4e79a7f9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 14:52:21 +0300 Subject: [PATCH 022/950] Trying style check --- .github/workflows/hello-world.yml | 9 +- tests/ci/report.py | 298 ++++++++++++++++++++++++++++++ tests/ci/style_check.py | 64 +++++++ 3 files changed, 368 insertions(+), 3 deletions(-) create mode 100644 tests/ci/report.py create mode 100644 tests/ci/style_check.py diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 9ef1c19fd3a..36bd25c8ad3 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -18,10 +18,13 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." - - run: mkdir -p $RUNNER_TEMP/style_check_result - - run: docker run --cap-add=SYS_PTRACE --volume=$GITHUB_WORKSPACE:/ClickHouse --volume=$RUNNER_TEMP/style_check_result:/test_output clickhouse/style-test:latest + - run: cd $GITHUB_WORKSPACE/test/ci && python3 style_check.py - name: List files in the repository run: | ls ${{ github.workspace }} - ls $RUNNER_TEMP/style_check_result + ls $RUNNER_TEMP + - uses: actions/upload-artifact@v2 + with: + name: report + path: $RUNNER_TEMP/report.html - run: echo "🍏 This job's status is ${{ job.status }}." diff --git a/tests/ci/report.py b/tests/ci/report.py new file mode 100644 index 00000000000..94defcfd648 --- /dev/null +++ b/tests/ci/report.py @@ -0,0 +1,298 @@ +# -*- coding: utf-8 -*- +import os +import datetime + +### FIXME: BEST FRONTEND PRACTICIES BELOW + +HTML_BASE_TEST_TEMPLATE = """ + + + + {title} + + +
+ +

{header}

+ +{test_part} + + + +""" + +HTML_TEST_PART = """ + + +{headers} + +{rows} +
+""" + +BASE_HEADERS = ['Test name', 'Test status'] + + +def _format_header(header, branch_name, branch_url=None): + result = ' '.join([w.capitalize() for w in header.split(' ')]) + result = result.replace("Clickhouse", "ClickHouse") + result = result.replace("clickhouse", "ClickHouse") + if 'ClickHouse' not in result: + result = 'ClickHouse ' + result + result += ' for ' + if branch_url: + result += '{name}'.format(url=branch_url, name=branch_name) + else: + result += branch_name + return result + + +def _get_status_style(status): + style = "font-weight: bold;" + if status in ('OK', 'success', 'PASSED'): + style += 'color: #0A0;' + elif status in ('FAIL', 'failure', 'error', 'FAILED', 'Timeout'): + style += 'color: #F00;' + else: + style += 'color: #FFB400;' + return style + + +def _get_html_url(url): + if isinstance(url, str): + return '{name}'.format(url=url, name=os.path.basename(url)) + if isinstance(url, tuple): + return '{name}'.format(url=url[0], name=url[1]) + return '' + + +def create_test_html_report(header, test_result, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls=[]): + if test_result: + rows_part = "" + num_fails = 0 + has_test_time = False + has_test_logs = False + for result in test_result: + test_name = result[0] + test_status = result[1] + + test_logs = None + test_time = None + if len(result) > 2: + test_time = result[2] + has_test_time = True + + if len(result) > 3: + test_logs = result[3] + has_test_logs = True + + row = "" + row += "" + test_name + "" + style = _get_status_style(test_status) + + # Allow to quickly scroll to the first failure. + is_fail = test_status == "FAIL" or test_status == 'FLAKY' + is_fail_id = "" + if is_fail: + num_fails = num_fails + 1 + is_fail_id = 'id="fail' + str(num_fails) + '" ' + + row += ''.format(style) + test_status + "" + + if test_time is not None: + row += "" + test_time + "" + + if test_logs is not None: + test_logs_html = "
".join([_get_html_url(url) for url in test_logs]) + row += "" + test_logs_html + "" + + row += "" + rows_part += row + + headers = BASE_HEADERS + if has_test_time: + headers.append('Test time, sec.') + if has_test_logs: + headers.append('Logs') + + headers = ''.join(['' + h + '' for h in headers]) + test_part = HTML_TEST_PART.format(headers=headers, rows=rows_part) + else: + test_part = "" + + additional_html_urls = "" + for url in additional_urls: + additional_html_urls += ' ' + _get_html_url(url) + + result = HTML_BASE_TEST_TEMPLATE.format( + title=_format_header(header, branch_name), + header=_format_header(header, branch_name, branch_url), + raw_log_name=os.path.basename(raw_log_url), + raw_log_url=raw_log_url, + task_url=task_url, + test_part=test_part, + branch_name=branch_name, + commit_url=commit_url, + additional_urls=additional_html_urls + ) + return result + + +HTML_BASE_BUILD_TEMPLATE = """ + + + + +{title} + + +
+

{header}

+ + + + + + + + + + + + +{rows} +
CompilerBuild typeSanitizerBundledSplittedStatusBuild logBuild timeArtifacts
+ + + +""" + +LINK_TEMPLATE = '{text}' + + +def create_build_html_report(header, build_results, build_logs_urls, artifact_urls_list, task_url, branch_url, branch_name, commit_url): + rows = "" + for (build_result, build_log_url, artifact_urls) in zip(build_results, build_logs_urls, artifact_urls_list): + row = "" + row += "{}".format(build_result.compiler) + if build_result.build_type: + row += "{}".format(build_result.build_type) + else: + row += "{}".format("relwithdebuginfo") + if build_result.sanitizer: + row += "{}".format(build_result.sanitizer) + else: + row += "{}".format("none") + + row += "{}".format(build_result.bundled) + row += "{}".format(build_result.splitted) + + if build_result.status: + style = _get_status_style(build_result.status) + row += '{}'.format(style, build_result.status) + else: + style = _get_status_style("error") + row += '{}'.format(style, "error") + + row += 'link'.format(build_log_url) + + if build_result.elapsed_seconds: + delta = datetime.timedelta(seconds=build_result.elapsed_seconds) + else: + delta = 'unknown' + + row += '{}'.format(str(delta)) + + links = "" + link_separator = "
" + if artifact_urls: + for artifact_url in artifact_urls: + links += LINK_TEMPLATE.format(text=os.path.basename(artifact_url), url=artifact_url) + links += link_separator + if links: + links = links[:-len(link_separator)] + row += "{}".format(links) + + row += "" + rows += row + return HTML_BASE_BUILD_TEMPLATE.format( + title=_format_header(header, branch_name), + header=_format_header(header, branch_name, branch_url), + rows=rows, + task_url=task_url, + branch_name=branch_name, + commit_url=commit_url) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py new file mode 100644 index 00000000000..671bd2c6893 --- /dev/null +++ b/tests/ci/style_check.py @@ -0,0 +1,64 @@ +#!/usr/bin/env python3 +from report import create_test_html_report +import logging +import subprocess +import os +import csv + + +def process_result(result_folder): + test_results = [] + additional_files = [] + # Just upload all files from result_folder. + # If task provides processed results, then it's responsible for content of result_folder. + if os.path.exists(result_folder): + test_files = [f for f in os.listdir(result_folder) if os.path.isfile(os.path.join(result_folder, f))] + additional_files = [os.path.join(result_folder, f) for f in test_files] + + status_path = os.path.join(result_folder, "check_status.tsv") + logging.info("Found test_results.tsv") + status = list(csv.reader(open(status_path, 'r'), delimiter='\t')) + if len(status) != 1 or len(status[0]) != 2: + return "error", "Invalid check_status.tsv", test_results, additional_files + state, description = status[0][0], status[0][1] + + try: + results_path = os.path.join(result_folder, "test_results.tsv") + test_results = list(csv.reader(open(results_path, 'r'), delimiter='\t')) + if len(test_results) == 0: + raise Exception("Empty results") + + return state, description, test_results, additional_files + except Exception: + if state == "success": + state, description = "error", "Failed to read test_results.tsv" + return state, description, test_results, additional_files + +def get_pr_url_from_ref(ref): + try: + return ref.split("/")[2] + except: + return "master" + +if __name__ == "__main__": + repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) + temp_path = os.getenv("RUNNER_TEMP", os.path.abspath("./temp")) + run_id = os.getenv("GITHUB_RUN_ID", 0) + commit_sha = os.getenv("GITHUB_SHA", 0) + ref = os.getenv("GITHUB_REF", "") + docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) + state, description, test_results, additional_files = process_result(temp_path) + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{run_id}" + branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(get_pr_url_from_ref(ref)) + branch_name = "PR #" + str(get_pr_url_from_ref(ref)) + commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" + raw_log_url = "noop" + + html_report = create_test_html_report("Style Check (actions)", test_results, raw_log_url, task_url, branch_url, branch_name, commit_url) + with open(os.path.join(temp_path, 'report.html'), 'w') as f: + f.write(html_report) From 2931810dfa39aad1994bebccc4c7318d4377ea29 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 14:53:34 +0300 Subject: [PATCH 023/950] Fix --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 36bd25c8ad3..3868dfe0cad 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -18,7 +18,7 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." - - run: cd $GITHUB_WORKSPACE/test/ci && python3 style_check.py + - run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - name: List files in the repository run: | ls ${{ github.workspace }} From 499e713959edae2281f018389705d3749d0e7979 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:12:47 +0300 Subject: [PATCH 024/950] Trying other way --- .github/workflows/hello-world.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 3868dfe0cad..724d1d742cc 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,5 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: $RUNNER_TEMP/report.html + path: ${{ env.RUNNER_TEMP }}/report.html + - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." From 53fe271c2ef1f1525a2f81bb9573c7f8fc419e05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:15:31 +0300 Subject: [PATCH 025/950] One more time --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 724d1d742cc..ed822c32d40 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,6 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: ${{ env.RUNNER_TEMP }}/report.html + path: $RUNNER_TEMP/report.html - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." From 1991e0a52836cc198829fbfbcc53ecc518f332d7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:18:58 +0300 Subject: [PATCH 026/950] One more --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index ed822c32d40..2917a6bb31b 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,6 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: $RUNNER_TEMP/report.html + path: $GITHUB_WORKSPACE/tests/ci/style_check.py - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." From ee32c34d9a94901e1af6393b7a3ddd7aa21053b2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:23:04 +0300 Subject: [PATCH 027/950] Something wrong --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 2917a6bb31b..6c1c8e0dd85 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,6 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: $GITHUB_WORKSPACE/tests/ci/style_check.py + path: ${{ github.workspace }}/tests/ci/style_check.py - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." From 686f8b4f09a24ee4b8b5e31274a15eef56c1fc36 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:40:13 +0300 Subject: [PATCH 028/950] One more try --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 6c1c8e0dd85..a81cc31fff0 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,6 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: ${{ github.workspace }}/tests/ci/style_check.py + path: ${{ runner.temp }}/report.hml - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." From e8c0f357080eaed23671b4a5b2801f65b9fa8f75 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:43:55 +0300 Subject: [PATCH 029/950] Report html --- .github/workflows/hello-world.yml | 2 +- tests/ci/style_check.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index a81cc31fff0..8ef331a2564 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,6 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: ${{ runner.temp }}/report.hml + path: report.hml - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 671bd2c6893..9fd55b372c2 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -60,5 +60,5 @@ if __name__ == "__main__": raw_log_url = "noop" html_report = create_test_html_report("Style Check (actions)", test_results, raw_log_url, task_url, branch_url, branch_name, commit_url) - with open(os.path.join(temp_path, 'report.html'), 'w') as f: + with open('report.html', 'w') as f: f.write(html_report) From 03c6a31e7c1528cb3b10cbedc6c21c17bc753b2f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:49:52 +0300 Subject: [PATCH 030/950] Fix --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 8ef331a2564..c1ba922ff92 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -26,6 +26,6 @@ jobs: - uses: actions/upload-artifact@v2 with: name: report - path: report.hml + path: ${{ runner.temp }}/report.html - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." From bb778cc0fe62be23635a3c60d719fedd68bd301d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 15:50:24 +0300 Subject: [PATCH 031/950] Followup --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9fd55b372c2..671bd2c6893 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -60,5 +60,5 @@ if __name__ == "__main__": raw_log_url = "noop" html_report = create_test_html_report("Style Check (actions)", test_results, raw_log_url, task_url, branch_url, branch_name, commit_url) - with open('report.html', 'w') as f: + with open(os.path.join(temp_path, 'report.html'), 'w') as f: f.write(html_report) From 7538f6f1686bd06e3b065ab69e88311b2a790bfd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 17:27:03 +0300 Subject: [PATCH 032/950] Better stylecheck --- .github/workflows/hello-world.yml | 7 ++- tests/ci/compress_files.py | 51 ++++++++++++++++ tests/ci/s3_helper.py | 99 +++++++++++++++++++++++++++++++ tests/ci/style_check.py | 57 +++++++++++++++--- 4 files changed, 204 insertions(+), 10 deletions(-) create mode 100644 tests/ci/compress_files.py create mode 100644 tests/ci/s3_helper.py diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index c1ba922ff92..155e9487ff4 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -1,4 +1,5 @@ name: GitHub Actions Hello self hosted +desction: Trying GithubActions on: push: branches: @@ -18,7 +19,11 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." - - run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py + - name: Style Check + env: + YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} + YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - name: List files in the repository run: | ls ${{ github.workspace }} diff --git a/tests/ci/compress_files.py b/tests/ci/compress_files.py new file mode 100644 index 00000000000..f095b04872b --- /dev/null +++ b/tests/ci/compress_files.py @@ -0,0 +1,51 @@ +#!/usr/bin/env python3 +import subprocess +import logging +import os + +def compress_file_fast(path, archive_path): + if os.path.exists('/usr/bin/pigz'): + subprocess.check_call("pigz < {} > {}".format(path, archive_path), shell=True) + else: + subprocess.check_call("gzip < {} > {}".format(path, archive_path), shell=True) + + +def compress_fast(path, archive_path, exclude=None): + pigz_part = '' + if os.path.exists('/usr/bin/pigz'): + logging.info("pigz found, will compress and decompress faster") + pigz_part = "--use-compress-program='pigz'" + else: + pigz_part = '-z' + logging.info("no pigz, compressing with default tar") + + if exclude is None: + exclude_part = "" + elif isinstance(exclude, list): + exclude_part = " ".join(["--exclude {}".format(x) for x in exclude]) + else: + exclude_part = "--exclude {}".format(str(exclude)) + + fname = os.path.basename(path) + if os.path.isfile(path): + path = os.path.dirname(path) + else: + path += "/.." + cmd = "tar {} {} -cf {} -C {} {}".format(pigz_part, exclude_part, archive_path, path, fname) + logging.debug("compress_fast cmd:{}".format(cmd)) + subprocess.check_call(cmd, shell=True) + + +def decompress_fast(archive_path, result_path=None): + pigz_part = '' + if os.path.exists('/usr/bin/pigz'): + logging.info("pigz found, will compress and decompress faster ('{}' -> '{}')".format(archive_path, result_path)) + pigz_part = "--use-compress-program='pigz'" + else: + pigz_part = '-z' + logging.info("no pigz, decompressing with default tar ('{}' -> '{}')".format(archive_path, result_path)) + + if result_path is None: + subprocess.check_call("tar {} -xf {}".format(pigz_part, archive_path), shell=True) + else: + subprocess.check_call("tar {} -xf {} -C {}".format(pigz_part, archive_path, result_path), shell=True) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py new file mode 100644 index 00000000000..8a170da44f8 --- /dev/null +++ b/tests/ci/s3_helper.py @@ -0,0 +1,99 @@ +# -*- coding: utf-8 -*- +import hashlib +import logging +import os +import boto3 +from botocore.exceptions import ClientError, BotoCoreError +from multiprocessing.dummy import Pool +from compress_files import compress_file_fast + +def _md5(fname): + hash_md5 = hashlib.md5() + with open(fname, "rb") as f: + for chunk in iter(lambda: f.read(4096), b""): + hash_md5.update(chunk) + logging.debug("MD5 for {} is {}".format(fname, hash_md5.hexdigest())) + return hash_md5.hexdigest() + + +def _flatten_list(lst): + result = [] + for elem in lst: + if isinstance(elem, list): + result += _flatten_list(elem) + else: + result.append(elem) + return result + + +class S3Helper(object): + def __init__(self, host, aws_access_key_id, aws_secret_access_key): + self.session = boto3.session.Session(aws_access_key_id=aws_access_key_id, aws_secret_access_key=aws_secret_access_key) + self.client = self.session.client('s3', endpoint_url=host) + + def _upload_file_to_s3(self, bucket_name, file_path, s3_path): + logging.debug("Start uploading {} to bucket={} path={}".format(file_path, bucket_name, s3_path)) + metadata = {} + if os.path.getsize(file_path) < 64 * 1024 * 1024: + if s3_path.endswith("txt") or s3_path.endswith("log") or s3_path.endswith("err") or s3_path.endswith("out"): + metadata['ContentType'] = "text/plain; charset=utf-8" + logging.info("Content type %s for file path %s", "text/plain; charset=utf-8", file_path) + elif s3_path.endswith("html"): + metadata['ContentType'] = "text/html; charset=utf-8" + logging.info("Content type %s for file path %s", "text/html; charset=utf-8", file_path) + else: + logging.info("No content type provied for %s", file_path) + else: + if s3_path.endswith("txt") or s3_path.endswith("log") or s3_path.endswith("err") or s3_path.endswith("out"): + logging.info("Going to compress file log file %s to %s", file_path, file_path + ".gz") + compress_file_fast(file_path, file_path + ".gz") + file_path += ".gz" + s3_path += ".gz" + else: + logging.info("Processing file without compression") + logging.info("File is too large, do not provide content type") + + self.client.upload_file(file_path, bucket_name, s3_path, ExtraArgs=metadata) + logging.info("Upload {} to {}. Meta: {}".format(file_path, s3_path, metadata)) + return "https://storage.yandexcloud.net/{bucket}/{path}".format(bucket=bucket_name, path=s3_path) + + def upload_test_report_to_s3(self, file_path, s3_path): + return self._upload_file_to_s3('clickhouse-test-reports', file_path, s3_path) + + def upload_build_file_to_s3(self, file_path, s3_path): + return self._upload_file_to_s3('clickhouse-builds', file_path, s3_path) + + def _upload_folder_to_s3(self, folder_path, s3_folder_path, bucket_name, keep_dirs_in_s3_path, upload_symlinks): + logging.info("Upload folder '{}' to bucket={} of s3 folder '{}'".format(folder_path, bucket_name, s3_folder_path)) + if not os.path.exists(folder_path): + return [] + files = os.listdir(folder_path) + if not files: + return [] + + p = Pool(min(len(files), 30)) + + def task(file_name): + full_fs_path = os.path.join(folder_path, file_name) + if keep_dirs_in_s3_path: + full_s3_path = s3_folder_path + "/" + os.path.basename(folder_path) + else: + full_s3_path = s3_folder_path + + if os.path.isdir(full_fs_path): + return self._upload_folder_to_s3(full_fs_path, full_s3_path, bucket_name, keep_dirs_in_s3_path, upload_symlinks) + + if os.path.islink(full_fs_path): + if upload_symlinks: + return self._upload_file_to_s3(bucket_name, full_fs_path, full_s3_path + "/" + file_name) + return [] + + return self._upload_file_to_s3(bucket_name, full_fs_path, full_s3_path + "/" + file_name) + + return sorted(_flatten_list(list(p.map(task, files)))) + + def upload_build_folder_to_s3(self, folder_path, s3_folder_path, keep_dirs_in_s3_path=True, upload_symlinks=True): + return self._upload_folder_to_s3(folder_path, s3_folder_path, 'clickhouse-builds', keep_dirs_in_s3_path, upload_symlinks) + + def upload_test_folder_to_s3(self, folder_path, s3_folder_path): + return self._upload_folder_to_s3(folder_path, s3_folder_path, 'clickhouse-test-reports', True, True) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 671bd2c6893..05274e78386 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -4,6 +4,19 @@ import logging import subprocess import os import csv +from s3_helper import S3Helper + + +def process_logs(s3_client, additional_logs, s3_path_prefix): + additional_urls = [] + for log_path in additional_logs: + if log_path: + additional_urls.append( + s3_client.upload_test_report_to_s3( + log_path, + s3_path_prefix + "/" + os.path.basename(log_path))) + + return additional_urls def process_result(result_folder): @@ -34,6 +47,31 @@ def process_result(result_folder): state, description = "error", "Failed to read test_results.tsv" return state, description, test_results, additional_files +def upload_results(s3_client, pr_number, commit_sha, state, description, test_results, additional_files): + s3_path_prefix = f"{pr_number}/{commit_sha}/style_check" + additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) + + # Add link to help. Anchors in the docs must be adjusted accordingly. + branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_name = "master" + if pr_number != 0: + branch_name = "PR #{}".format(pr_number) + branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(pr_number) + commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" + + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{run_id}" + + raw_log_url = additional_urls[0] + additional_urls.pop(0) + + html_report = create_test_html_report("Style Check (actions)", test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + with open('report.html', 'w') as f: + f.write(html_report) + + url = s3_client.upload_test_report_to_s3('report.html', s3_path_prefix + ".html") + logging.info("Search result in url %s", url) + + def get_pr_url_from_ref(ref): try: return ref.split("/")[2] @@ -41,24 +79,25 @@ def get_pr_url_from_ref(ref): return "master" if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) temp_path = os.getenv("RUNNER_TEMP", os.path.abspath("./temp")) run_id = os.getenv("GITHUB_RUN_ID", 0) commit_sha = os.getenv("GITHUB_SHA", 0) ref = os.getenv("GITHUB_REF", "") + aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") + aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") + docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") + if not aws_secret_key_id or not aws_secret_key: + logging.info("No secrets, will not upload anything to S3") + + s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) if not os.path.exists(temp_path): os.makedirs(temp_path) subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) - state, description, test_results, additional_files = process_result(temp_path) - task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{run_id}" - branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(get_pr_url_from_ref(ref)) - branch_name = "PR #" + str(get_pr_url_from_ref(ref)) - commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" - raw_log_url = "noop" - html_report = create_test_html_report("Style Check (actions)", test_results, raw_log_url, task_url, branch_url, branch_name, commit_url) - with open(os.path.join(temp_path, 'report.html'), 'w') as f: - f.write(html_report) + state, description, test_results, additional_files = process_result(temp_path) + upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) From f14ee387d55fe7bd734c258d10c7c0a6b738762c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 17:28:05 +0300 Subject: [PATCH 033/950] Fix --- .github/workflows/hello-world.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 155e9487ff4..f8c5499fddd 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -1,5 +1,5 @@ name: GitHub Actions Hello self hosted -desction: Trying GithubActions +description: Trying GithubActions on: push: branches: From d353fd1a3d65655a97bde7cd16cbd566fdf9ada5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 17:28:57 +0300 Subject: [PATCH 034/950] Remove description --- .github/workflows/hello-world.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index f8c5499fddd..bb89fd7bea7 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -1,5 +1,4 @@ name: GitHub Actions Hello self hosted -description: Trying GithubActions on: push: branches: From efaf9e758350027f069e6dd70d98b687a3325925 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Sep 2021 17:38:34 +0300 Subject: [PATCH 035/950] Upload from separate dir --- .github/workflows/hello-world.yml | 4 ---- tests/ci/style_check.py | 6 +++++- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index bb89fd7bea7..de2419ea506 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -27,9 +27,5 @@ jobs: run: | ls ${{ github.workspace }} ls $RUNNER_TEMP - - uses: actions/upload-artifact@v2 - with: - name: report - path: ${{ runner.temp }}/report.html - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 05274e78386..e527baecfe5 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 from report import create_test_html_report +import shutil import logging import subprocess import os @@ -81,7 +82,7 @@ def get_pr_url_from_ref(ref): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) - temp_path = os.getenv("RUNNER_TEMP", os.path.abspath("./temp")) + temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'style_check') run_id = os.getenv("GITHUB_RUN_ID", 0) commit_sha = os.getenv("GITHUB_SHA", 0) ref = os.getenv("GITHUB_REF", "") @@ -94,6 +95,9 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + if os.path.exists(temp_path): + shutil.rmtree(temp_path) + if not os.path.exists(temp_path): os.makedirs(temp_path) From 810d180ce2faf08b2e2e0eb1d1dcf00777194b31 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 10 Sep 2021 17:21:03 +0000 Subject: [PATCH 036/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 88 ++++++++++++------------- src/Storages/FileLog/StorageFileLog.h | 5 +- 2 files changed, 45 insertions(+), 48 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ba64a590e63..40c1a780739 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -83,6 +82,8 @@ StorageFileLog::StorageFileLog( throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); } + directory_watch = std::make_unique(path); + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); @@ -207,7 +208,7 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activateAndSchedule(); + watch_task->activate(); // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) @@ -215,6 +216,8 @@ void StorageFileLog::threadFunc() if (!checkDependencies(table_id)) break; + watch_task->scheduleAfter(RESCHEDULE_MS); + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); auto stream_is_stalled = streamToViews(); @@ -319,15 +322,15 @@ void StorageFileLog::clearInvalidFiles() /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function std::vector valid_files; - for (const auto & it : file_names) + for (const auto & file_name : file_names) { - if (file_status.at(it).status == FileStatus::REMOVED) + if (file_status.at(file_name).status == FileStatus::REMOVED) { - file_status.erase(it); + file_status.erase(file_name); } else { - valid_files.push_back(it); + valid_files.push_back(file_name); } } @@ -394,52 +397,43 @@ void registerStorageFileLog(StorageFactory & factory) void StorageFileLog::watchFunc() { - FileLogDirectoryWatcher dw(path); - while (true) + auto error = directory_watch->hasError(); + if (error) + LOG_INFO(log, "Error happened during watching directory {}.", directory_watch->getPath()); + + auto events = directory_watch->getEvents(); + + for (const auto & event : events) { - sleepForMilliseconds(getPollTimeoutMillisecond()); - - auto error = dw.hasError(); - if (error) - LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); - - auto events = dw.getEvents(); - - for (const auto & event : events) + std::lock_guard lock(status_mutex); + switch (event.type) { - std::lock_guard lock(status_mutex); - switch (event.type) - { + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path)) + { + file_status[event.path].reader = std::ifstream(event.path); + file_names.push_back(event.path); + } + break; - case Poco::DirectoryWatcher::DW_ITEM_ADDED: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path)) - { - file_status[event.path].reader = std::ifstream(event.path); - file_names.push_back(event.path); - } - break; + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::UPDATED; + } + break; - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { - file_status[event.path].status = FileStatus::UPDATED; - } - break; - - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { - file_status[event.path].status = FileStatus::REMOVED; - } - break; - } + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::REMOVED; + } } } } - } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ada9ebf4004..a3f830a086a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -83,6 +84,8 @@ private: std::mutex status_mutex; + std::unique_ptr directory_watch = nullptr; + // Stream thread struct TaskContext { @@ -109,7 +112,7 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - [[noreturn]] void watchFunc(); + void watchFunc(); }; } From aa9951638016eeeb9b3673f5c40f5a18cd9e81e5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 12 Sep 2021 16:25:15 +0000 Subject: [PATCH 037/950] fix fix --- src/Storages/FileLog/FileLogSource.cpp | 13 ++++- src/Storages/FileLog/StorageFileLog.cpp | 70 +++++++++++-------------- src/Storages/FileLog/StorageFileLog.h | 4 +- 3 files changed, 44 insertions(+), 43 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index ecc43c579c7..61410e8488c 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -15,6 +15,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +const auto MAX_FAILED_POLL_ATTEMPTS = 10; + FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -91,10 +93,11 @@ Chunk FileLogSource::generate() }; size_t total_rows = 0; + size_t failed_poll_attempts = 0; + Stopwatch watch; while (true) { - Stopwatch watch; size_t new_rows = 0; if (buffer->poll()) { @@ -111,8 +114,14 @@ Chunk FileLogSource::generate() { total_rows = total_rows + new_rows; } + else + { + ++failed_poll_attempts; + } - if (!buffer->hasMorePolledRecords() && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out)) + if (!buffer->hasMorePolledRecords() + && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out + || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) { break; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 40c1a780739..c2012d40208 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -84,8 +84,6 @@ StorageFileLog::StorageFileLog( directory_watch = std::make_unique(path); - watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); } @@ -101,17 +99,18 @@ Pipe StorageFileLog::read( { std::lock_guard lock(status_mutex); - auto modified_context = Context::createCopy(local_context); + updateFileStatus(); - clearInvalidFiles(); - - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); /// No files to parse - if (max_streams_number == 0) + if (file_names.empty()) { return Pipe{}; } + auto modified_context = Context::createCopy(local_context); + + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) @@ -142,7 +141,6 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); - watch_task->deactivate(); for (auto & file : file_status) { @@ -208,20 +206,16 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activate(); - // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) { if (!checkDependencies(table_id)) break; - watch_task->scheduleAfter(RESCHEDULE_MS); - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - auto stream_is_stalled = streamToViews(); - if (stream_is_stalled) + auto file_status_no_update = streamToViews(); + if (file_status_no_update) { LOG_TRACE(log, "Stream stalled. Reschedule."); break; @@ -234,6 +228,7 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } + updateFileStatus(); } } } @@ -242,7 +237,6 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -260,9 +254,7 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - clearInvalidFiles(); - - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); /// No files to parse if (max_streams_number == 0) { @@ -280,13 +272,14 @@ bool StorageFileLog::streamToViews() Pipes pipes; pipes.reserve(max_streams_number); + auto names = block_io.out->getHeader().getNames(); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { pipes.emplace_back(std::make_shared( *this, metadata_snapshot, new_context, - block_io.out->getHeader().getNames(), + names, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, @@ -314,27 +307,11 @@ bool StorageFileLog::streamToViews() LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return true; + return updateFileStatus(); } void StorageFileLog::clearInvalidFiles() { - /// Do not need to hold file_status lock, since it will be holded - /// by caller when call this function - std::vector valid_files; - for (const auto & file_name : file_names) - { - if (file_status.at(file_name).status == FileStatus::REMOVED) - { - file_status.erase(file_name); - } - else - { - valid_files.push_back(file_name); - } - } - - file_names.swap(valid_files); } void registerStorageFileLog(StorageFactory & factory) @@ -395,7 +372,7 @@ void registerStorageFileLog(StorageFactory & factory) }); } -void StorageFileLog::watchFunc() +bool StorageFileLog::updateFileStatus() { auto error = directory_watch->hasError(); if (error) @@ -405,7 +382,6 @@ void StorageFileLog::watchFunc() for (const auto & event : events) { - std::lock_guard lock(status_mutex); switch (event.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: @@ -435,5 +411,23 @@ void StorageFileLog::watchFunc() } } } + /// Do not need to hold file_status lock, since it will be holded + /// by caller when call this function + std::vector valid_files; + for (const auto & file_name : file_names) + { + if (file_status.at(file_name).status == FileStatus::REMOVED) + { + file_status.erase(file_name); + } + else + { + valid_files.push_back(file_name); + } + } + + file_names.swap(valid_files); + + return events.empty() || file_names.empty(); } } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index a3f830a086a..9f856fb26ea 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -99,8 +99,6 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; - TaskThread watch_task; - void threadFunc(); void clearInvalidFiles(); @@ -112,7 +110,7 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - void watchFunc(); + bool updateFileStatus(); }; } From bd1990a67c3fe28c478d4531b1ec114407760b21 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 03:44:12 +0000 Subject: [PATCH 038/950] fix test and restrict path --- src/Storages/FileLog/StorageFileLog.cpp | 6 +++--- src/Storages/FileLog/StorageFileLog.h | 2 +- tests/queries/0_stateless/01645_system_table_engines.sql | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c2012d40208..96745b5f454 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -45,13 +45,13 @@ StorageFileLog::StorageFileLog( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & path_, + const String & relative_path_, const String & format_name_, std::unique_ptr settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) - , path(path_) + , path(getContext()->getUserFilesPath() + "/" + relative_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) { @@ -254,7 +254,7 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); /// No files to parse if (max_streams_number == 0) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 9f856fb26ea..e6d466e921b 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -59,7 +59,7 @@ protected: const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & path_, + const String & relative_path_, const String & format_name_, std::unique_ptr settings); diff --git a/tests/queries/0_stateless/01645_system_table_engines.sql b/tests/queries/0_stateless/01645_system_table_engines.sql index 5e8eef5508b..c2048427670 100644 --- a/tests/queries/0_stateless/01645_system_table_engines.sql +++ b/tests/queries/0_stateless/01645_system_table_engines.sql @@ -1 +1 @@ -SELECT * FROM system.table_engines WHERE name in ('MergeTree', 'ReplicatedCollapsingMergeTree') FORMAT PrettyCompactNoEscapes; +SELECT * FROM system.table_engines WHERE name in ('MergeTree', 'ReplicatedCollapsingMergeTree') ORDER BY name FORMAT PrettyCompactNoEscapes; From 9d9ba5680827ea65b3da241e9af2dc78b2a090c4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 04:46:47 +0000 Subject: [PATCH 039/950] add test remove unneeded code --- .../FileLog/ReadBufferFromFileLog.cpp | 4 -- .../test_storage_filelog/__init__.py | 0 .../configs/config.d/cluster.xml | 16 +++++ .../configs/user_files_path.xml | 3 + .../integration/test_storage_filelog/test.py | 67 +++++++++++++++++++ .../user_files/logs/data.csv | 10 +++ 6 files changed, 96 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_storage_filelog/__init__.py create mode 100644 tests/integration/test_storage_filelog/configs/config.d/cluster.xml create mode 100644 tests/integration/test_storage_filelog/configs/user_files_path.xml create mode 100644 tests/integration/test_storage_filelog/test.py create mode 100644 tests/integration/test_storage_filelog/user_files/logs/data.csv diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 0c164c6b070..8e1d16ba8d7 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -11,10 +11,6 @@ namespace DB { -namespace ErrorCodes -{ -} - ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, size_t max_batch_size, diff --git a/tests/integration/test_storage_filelog/__init__.py b/tests/integration/test_storage_filelog/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_filelog/configs/config.d/cluster.xml b/tests/integration/test_storage_filelog/configs/config.d/cluster.xml new file mode 100644 index 00000000000..96e0e7c34c1 --- /dev/null +++ b/tests/integration/test_storage_filelog/configs/config.d/cluster.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9001 + + + + + diff --git a/tests/integration/test_storage_filelog/configs/user_files_path.xml b/tests/integration/test_storage_filelog/configs/user_files_path.xml new file mode 100644 index 00000000000..b41f0f8e00d --- /dev/null +++ b/tests/integration/test_storage_filelog/configs/user_files_path.xml @@ -0,0 +1,3 @@ + + user_files + diff --git a/tests/integration/test_storage_filelog/test.py b/tests/integration/test_storage_filelog/test.py new file mode 100644 index 00000000000..9940c0afa7a --- /dev/null +++ b/tests/integration/test_storage_filelog/test.py @@ -0,0 +1,67 @@ +import pytest +import time +import shutil +import os +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', + main_configs=['configs/user_files_path.xml']) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_select(started_cluster): + try: + name = "filelog_table" + node1.query(""" + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + """.format(name=name)) + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" + + shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + os.remove("user_files/logs/data_copy.csv") + + +def test_stream_to_view(started_cluster): + try: + name = "filelog_table" + node1.query(""" + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + """.format(name=name)) + + target_name = "mv" + node1.query(""" + CREATE Materialized View {target_name} engine=MergeTree order by id as select * from {name} + """.format(target_name = target_name, name=name)) + + time.sleep(10) + + assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "55\n" + + shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") + + time.sleep(10) + + assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "110\n" + + finally: + node1.query("DROP TABLE IF EXISTS {target_name}".format(target_name=target_name)) + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + os.remove("user_files/logs/data_copy.csv") diff --git a/tests/integration/test_storage_filelog/user_files/logs/data.csv b/tests/integration/test_storage_filelog/user_files/logs/data.csv new file mode 100644 index 00000000000..dd198b1bc93 --- /dev/null +++ b/tests/integration/test_storage_filelog/user_files/logs/data.csv @@ -0,0 +1,10 @@ +1, 1 +2, 2 +3, 3 +4, 4 +5, 5 +6, 6 +7, 7 +8, 8 +9, 9 +10, 10 From d2192902bf3c92b8fbc42c1cddac09936bf465cf Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 06:16:43 +0000 Subject: [PATCH 040/950] try fix test --- tests/integration/test_storage_filelog/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_filelog/test.py b/tests/integration/test_storage_filelog/test.py index 9940c0afa7a..01e9e6e0d13 100644 --- a/tests/integration/test_storage_filelog/test.py +++ b/tests/integration/test_storage_filelog/test.py @@ -25,12 +25,12 @@ def test_select(started_cluster): try: name = "filelog_table" node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') """.format(name=name)) assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" - shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") + shutil.copy("user_files/logs/data.csv", "user_files/logs/data_copy.csv") assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" @@ -43,7 +43,7 @@ def test_stream_to_view(started_cluster): try: name = "filelog_table" node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') """.format(name=name)) target_name = "mv" From 218ac23a875c7e0cfd29d3bc45a9a2a332516802 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 11:09:31 +0000 Subject: [PATCH 041/950] update test fix test --- .../test_storage_filelog/__init__.py | 0 .../configs/config.d/cluster.xml | 16 ----- .../configs/user_files_path.xml | 3 - .../integration/test_storage_filelog/test.py | 67 ------------------- .../user_files/logs/data.csv | 10 --- .../02023_storage_filelog.reference | 67 +++++++++++++++++++ .../0_stateless/02023_storage_filelog.sh | 50 ++++++++++++++ 7 files changed, 117 insertions(+), 96 deletions(-) delete mode 100644 tests/integration/test_storage_filelog/__init__.py delete mode 100644 tests/integration/test_storage_filelog/configs/config.d/cluster.xml delete mode 100644 tests/integration/test_storage_filelog/configs/user_files_path.xml delete mode 100644 tests/integration/test_storage_filelog/test.py delete mode 100644 tests/integration/test_storage_filelog/user_files/logs/data.csv create mode 100644 tests/queries/0_stateless/02023_storage_filelog.reference create mode 100755 tests/queries/0_stateless/02023_storage_filelog.sh diff --git a/tests/integration/test_storage_filelog/__init__.py b/tests/integration/test_storage_filelog/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_filelog/configs/config.d/cluster.xml b/tests/integration/test_storage_filelog/configs/config.d/cluster.xml deleted file mode 100644 index 96e0e7c34c1..00000000000 --- a/tests/integration/test_storage_filelog/configs/config.d/cluster.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - - - node1 - 9000 - - - node2 - 9001 - - - - - diff --git a/tests/integration/test_storage_filelog/configs/user_files_path.xml b/tests/integration/test_storage_filelog/configs/user_files_path.xml deleted file mode 100644 index b41f0f8e00d..00000000000 --- a/tests/integration/test_storage_filelog/configs/user_files_path.xml +++ /dev/null @@ -1,3 +0,0 @@ - - user_files - diff --git a/tests/integration/test_storage_filelog/test.py b/tests/integration/test_storage_filelog/test.py deleted file mode 100644 index 01e9e6e0d13..00000000000 --- a/tests/integration/test_storage_filelog/test.py +++ /dev/null @@ -1,67 +0,0 @@ -import pytest -import time -import shutil -import os -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node1 = cluster.add_instance('node1', - main_configs=['configs/user_files_path.xml']) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_select(started_cluster): - try: - name = "filelog_table" - node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') - """.format(name=name)) - - assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" - - shutil.copy("user_files/logs/data.csv", "user_files/logs/data_copy.csv") - - assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" - - finally: - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) - os.remove("user_files/logs/data_copy.csv") - - -def test_stream_to_view(started_cluster): - try: - name = "filelog_table" - node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') - """.format(name=name)) - - target_name = "mv" - node1.query(""" - CREATE Materialized View {target_name} engine=MergeTree order by id as select * from {name} - """.format(target_name = target_name, name=name)) - - time.sleep(10) - - assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "55\n" - - shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") - - time.sleep(10) - - assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "110\n" - - finally: - node1.query("DROP TABLE IF EXISTS {target_name}".format(target_name=target_name)) - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) - os.remove("user_files/logs/data_copy.csv") diff --git a/tests/integration/test_storage_filelog/user_files/logs/data.csv b/tests/integration/test_storage_filelog/user_files/logs/data.csv deleted file mode 100644 index dd198b1bc93..00000000000 --- a/tests/integration/test_storage_filelog/user_files/logs/data.csv +++ /dev/null @@ -1,10 +0,0 @@ -1, 1 -2, 2 -3, 3 -4, 4 -5, 5 -6, 6 -7, 7 -8, 8 -9, 9 -10, 10 diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference new file mode 100644 index 00000000000..a1ac16d1d55 --- /dev/null +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -0,0 +1,67 @@ +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +100 100 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +100 100 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 +5 5 +5 5 +6 6 +6 6 +7 7 +7 7 +8 8 +8 8 +9 9 +9 9 +10 10 +10 10 +100 100 +100 100 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 +5 5 +5 5 +6 6 +6 6 +7 7 +7 7 +8 8 +8 8 +9 9 +9 9 +10 10 +10 10 +100 100 +100 100 +111 111 diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh new file mode 100755 index 00000000000..6cd40e64274 --- /dev/null +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +mkdir -p ${user_files_path}/logs/ +echo 1, 1 >> ${user_files_path}/logs/a.txt +echo 2, 2 >> ${user_files_path}/logs/a.txt +echo 3, 3 >> ${user_files_path}/logs/a.txt +echo 4, 4 >> ${user_files_path}/logs/a.txt +echo 5, 5 >> ${user_files_path}/logs/a.txt +echo 6, 6 >> ${user_files_path}/logs/a.txt +echo 7, 7 >> ${user_files_path}/logs/a.txt +echo 8, 8 >> ${user_files_path}/logs/a.txt +echo 9, 9 >> ${user_files_path}/logs/a.txt +echo 10, 10 >> ${user_files_path}/logs/a.txt + +### 1st TEST in CLIENT mode. +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +echo 100, 100 >> ${user_files_path}/logs/a.txt + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +echo 111, 111 >> ${user_files_path}/logs/a.txt + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +rm -rf ${user_files_path}/logs From cfa92a0c1e9639e7ead70b201f2db55fbeeadc96 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 14:50:27 +0000 Subject: [PATCH 042/950] fix fix --- .../FileLog/ReadBufferFromFileLog.cpp | 22 ++++++++++++---- src/Storages/FileLog/StorageFileLog.cpp | 25 ++++++++----------- src/Storages/FileLog/StorageFileLog.h | 2 +- .../0_stateless/02023_storage_filelog.sh | 4 +++ 4 files changed, 32 insertions(+), 21 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 8e1d16ba8d7..bb43afe08df 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -103,23 +103,35 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ for (size_t i = start; i < end; ++i) { auto & file = file_status[file_names[i]]; - if (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::REMOVED) + if (file.status == StorageFileLog::FileStatus::NO_CHANGE) continue; - while (read_records_size < need_records_size && file.reader.good() && !file.reader.eof()) + auto reader = std::ifstream(file_names[i]); + + reader.seekg(0, reader.end); + auto stream_end = reader.tellg(); + + reader.seekg(file.last_read_position); + + while (read_records_size < need_records_size && reader.tellg() < stream_end) { Record record; - std::getline(file.reader, record); + std::getline(reader, record); new_records.emplace_back(record); ++read_records_size; } - // Read to the end of the file - if (file.reader.eof()) + file.last_read_position = reader.tellg(); + + if (reader.tellg() == stream_end) + { file.status = StorageFileLog::FileStatus::NO_CHANGE; + } if (read_records_size == need_records_size) + { break; + } } } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 96745b5f454..91e6d045fce 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -61,7 +61,7 @@ StorageFileLog::StorageFileLog( if (std::filesystem::is_regular_file(path)) { - file_status[path].reader = std::ifstream(path); + file_status[path] = FileContext{}; file_names.push_back(path); } else if (std::filesystem::is_directory(path)) @@ -72,7 +72,7 @@ StorageFileLog::StorageFileLog( { if (dir_entry.is_regular_file()) { - file_status[dir_entry.path()].reader = std::ifstream(dir_entry.path()); + file_status[dir_entry.path()] = FileContext{}; file_names.push_back(dir_entry.path()); } } @@ -141,11 +141,6 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); - - for (auto & file : file_status) - { - file.second.reader.close(); - } } size_t StorageFileLog::getMaxBlockSize() const @@ -374,6 +369,8 @@ void registerStorageFileLog(StorageFactory & factory) bool StorageFileLog::updateFileStatus() { + /// Do not need to hold file_status lock, since it will be holded + /// by caller when call this function auto error = directory_watch->hasError(); if (error) LOG_INFO(log, "Error happened during watching directory {}.", directory_watch->getPath()); @@ -385,18 +382,18 @@ bool StorageFileLog::updateFileStatus() switch (event.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: - LOG_TRACE(log, "New event {} watched.", event.callback); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(event.path)) { - file_status[event.path].reader = std::ifstream(event.path); + file_status[event.path] = FileContext{}; file_names.push_back(event.path); } break; case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { file_status[event.path].status = FileStatus::UPDATED; } break; @@ -404,15 +401,13 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched.", event.callback); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) { file_status[event.path].status = FileStatus::REMOVED; } } } - /// Do not need to hold file_status lock, since it will be holded - /// by caller when call this function std::vector valid_files; for (const auto & file_name : file_names) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index e6d466e921b..d035c0e2903 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -74,7 +74,7 @@ private: struct FileContext { FileStatus status = FileStatus::BEGIN; - std::ifstream reader; + size_t last_read_position = 0; }; using NameToFile = std::unordered_map; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 6cd40e64274..4987a4b526b 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -41,10 +41,14 @@ ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt +sleep 5 + ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/a.txt +sleep 5 + ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" rm -rf ${user_files_path}/logs From ded27ec02fb337b9dd8584f5c1bf131a8dced5e4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 15:59:27 +0000 Subject: [PATCH 043/950] update update --- .../0_stateless/02023_storage_filelog.reference | 12 ++++++------ tests/queries/0_stateless/02023_storage_filelog.sh | 8 ++++++-- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference index a1ac16d1d55..f1aa3abf257 100644 --- a/tests/queries/0_stateless/02023_storage_filelog.reference +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -8,7 +8,7 @@ 8 8 9 9 10 10 -100 100 +100 100 1 1 2 2 3 3 @@ -19,7 +19,7 @@ 8 8 9 9 10 10 -100 100 +100 100 1 1 1 1 2 2 @@ -40,8 +40,8 @@ 9 9 10 10 10 10 -100 100 -100 100 +100 100 +100 100 1 1 1 1 2 2 @@ -62,6 +62,6 @@ 9 9 10 10 10 10 -100 100 -100 100 +100 100 +100 100 111 111 diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 4987a4b526b..3301c9c394a 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -36,19 +36,23 @@ cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" +${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt -sleep 5 +sleep 10 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/a.txt -sleep 5 +sleep 10 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" +${CLICKHOUSE_CLIENT} --query "drop table file_log;" +${CLICKHOUSE_CLIENT} --query "drop table mv;" + rm -rf ${user_files_path}/logs From 94ba18e050262570c51b94699dce3f5e481a0de4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 14 Sep 2021 03:15:05 +0000 Subject: [PATCH 044/950] fix --- src/Storages/FileLog/FileLogSource.cpp | 11 ++++++++++- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 11 ++++++++++- src/Storages/FileLog/ReadBufferFromFileLog.h | 5 ++++- 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 61410e8488c..08a8c4b6868 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -41,7 +41,7 @@ FileLogSource::FileLogSource( Chunk FileLogSource::generate() { - if (!buffer) + if (!buffer || buffer->noRecords()) return {}; MutableColumns read_columns = non_virtual_header.cloneEmptyColumns(); @@ -110,10 +110,19 @@ Chunk FileLogSource::generate() tryLogCurrentException(__PRETTY_FUNCTION__); } } + else + { + /// No records polled, should break out early, since + /// file status can not be updated during streamToViews + break; + } + if (new_rows) { total_rows = total_rows + new_rows; } + + /// poll succeed, but parse failed else { ++failed_poll_attempts; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index bb43afe08df..bf9cd2d9fce 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -76,13 +76,17 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz new_records.reserve(batch_size_); readNewRecords(new_records, batch_size); - if (new_records.size() == batch_size_) + if (new_records.size() == batch_size_ || stream_out) return new_records; Stopwatch watch; while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) { readNewRecords(new_records, batch_size); + /// All ifstrem reach end, no need to wait for timeout, + /// since file status can not be updated during a streamToViews + if (stream_out) + break; } return new_records; @@ -126,6 +130,11 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ if (reader.tellg() == stream_end) { file.status = StorageFileLog::FileStatus::NO_CHANGE; + /// All ifstream reach end + if (i == end - 1) + { + stream_out = true; + } } if (read_records_size == need_records_size) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index a7447e5edcd..66a8aeaa49f 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -40,16 +40,19 @@ public: private: enum class BufferStatus { + INIT, NO_RECORD_RETURNED, POLLED_OK, }; - BufferStatus buffer_status; + BufferStatus buffer_status = BufferStatus::INIT; Poco::Logger * log; StorageFileLog & storage; + bool stream_out = false; + size_t batch_size; size_t poll_timeout; From 19ed9c6dbc93ad6712abc2b468ceddd410980a44 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 14 Sep 2021 04:50:18 +0000 Subject: [PATCH 045/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- tests/queries/0_stateless/02023_storage_filelog.sh | 4 ++-- tests/queries/skip_list.json | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 91e6d045fce..eb95a094168 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -392,8 +392,8 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { file_status[event.path].status = FileStatus::UPDATED; } break; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 3301c9c394a..5a0333b1bc3 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -42,13 +42,13 @@ ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt -sleep 10 +sleep 5 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/a.txt -sleep 10 +sleep 5 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 0143cc78dbe..cb3be0c81b5 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -464,7 +464,6 @@ "memory_leak", "memory_limit", "polygon_dicts", // they use an explicitly specified database - "01658_read_file_to_stringcolumn", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. "01748_dictionary_table_dot", // creates database @@ -513,6 +512,7 @@ "01530_drop_database_atomic_sync", /// creates database "02001_add_default_database_to_system_users", ///create user "02002_row_level_filter_bug", ///create user - "02015_system_views" + "02015_system_views", + "02023_storage_filelog" ] } From 55d6c4e196986888031aca15493988134e8a3019 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 11:37:29 +0300 Subject: [PATCH 046/950] Trying to update check --- .github/workflows/hello-world.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index de2419ea506..1595e23a675 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -29,3 +29,9 @@ jobs: ls $RUNNER_TEMP - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." + - name: "Trying to update check link" + run: | + curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.run_id }} \ + --header 'authorization: Bearer ${{ secrets.GITHUB_TOKEN }}' \ + --header 'content-type: application/json' \ + -d '{"name" : "hello-world-name"}' From 2fa9c93b6b3c811dc5f206e1fe32875e6202463b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 11:39:15 +0300 Subject: [PATCH 047/950] Fix spaces --- .github/workflows/hello-world.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 1595e23a675..08e9599649e 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -31,7 +31,7 @@ jobs: - run: echo "🍏 This job's status is ${{ job.status }}." - name: "Trying to update check link" run: | - curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.run_id }} \ + curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.run_id }} \ --header 'authorization: Bearer ${{ secrets.GITHUB_TOKEN }}' \ --header 'content-type: application/json' \ - -d '{"name" : "hello-world-name"}' + -d '{"name" : "hello-world-name"}' From 2e3fad449ac9b908ab66d8f1a47dada3140df77f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 11:49:21 +0300 Subject: [PATCH 048/950] Trying more --- .github/workflows/hello-world.yml | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 08e9599649e..ab7cb75205d 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -18,11 +18,11 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." - - name: Style Check - env: - YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} - YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py + #- name: Style Check + # env: + # YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} + # YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + # run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - name: List files in the repository run: | ls ${{ github.workspace }} @@ -31,7 +31,13 @@ jobs: - run: echo "🍏 This job's status is ${{ job.status }}." - name: "Trying to update check link" run: | - curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.run_id }} \ + curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.job }} \ + --header 'authorization: Bearer ${{ secrets.GITHUB_TOKEN }}' \ + --header 'content-type: application/json' \ + -d '{"name" : "hello-world-name"}' + - name: "Trying to update check link" + run: | + curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.action }} \ --header 'authorization: Bearer ${{ secrets.GITHUB_TOKEN }}' \ --header 'content-type: application/json' \ -d '{"name" : "hello-world-name"}' From ebdd63aeca06d9bdb1ad0df04c3d478e335549cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:02:38 +0300 Subject: [PATCH 049/950] Trying other way --- .github/workflows/hello-world.yml | 25 +++++++------------------ tests/ci/style_check.py | 17 ++++++++++++++++- 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index ab7cb75205d..53fc1b64ff6 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -7,7 +7,7 @@ on: branches: - master jobs: - Explore-GitHub-Actions: + Style Check: runs-on: [self-hosted] steps: - run: echo "🎉 The job was automatically triggered by a ${{ github.event_name }} event." @@ -18,26 +18,15 @@ jobs: uses: actions/checkout@v2 - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - run: echo "🖥️ The workflow is now ready to test your code on the runner." - #- name: Style Check - # env: - # YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} - # YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - # run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py + - name: Style Check + env: + YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} + YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - name: List files in the repository run: | ls ${{ github.workspace }} ls $RUNNER_TEMP - run: ls -la $RUNNER_TEMP - run: echo "🍏 This job's status is ${{ job.status }}." - - name: "Trying to update check link" - run: | - curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.job }} \ - --header 'authorization: Bearer ${{ secrets.GITHUB_TOKEN }}' \ - --header 'content-type: application/json' \ - -d '{"name" : "hello-world-name"}' - - name: "Trying to update check link" - run: | - curl --request PATCH --url https://api.github.com/repos/${{ github.repository }}/check-runs/${{ github.action }} \ - --header 'authorization: Bearer ${{ secrets.GITHUB_TOKEN }}' \ - --header 'content-type: application/json' \ - -d '{"name" : "hello-world-name"}' diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index e527baecfe5..75fa1fefadf 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -1,4 +1,5 @@ #!/usr/bin/env python3 +from github import Github from report import create_test_html_report import shutil import logging @@ -7,6 +8,8 @@ import os import csv from s3_helper import S3Helper +NAME = "Style Check" + def process_logs(s3_client, additional_logs, s3_path_prefix): additional_urls = [] @@ -71,6 +74,7 @@ def upload_results(s3_client, pr_number, commit_sha, state, description, test_re url = s3_client.upload_test_report_to_s3('report.html', s3_path_prefix + ".html") logging.info("Search result in url %s", url) + return url def get_pr_url_from_ref(ref): @@ -79,6 +83,12 @@ def get_pr_url_from_ref(ref): except: return "master" +def get_check(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + check = list(commit.get_check_runs(NAME))[0] + return check + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) @@ -89,6 +99,10 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") + gh = Github(os.getenv("GITHUB_TOKEN")) + check = get_check(gh, commit_sha) + check.edit(name="Test style check") + docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") if not aws_secret_key_id or not aws_secret_key: logging.info("No secrets, will not upload anything to S3") @@ -104,4 +118,5 @@ if __name__ == "__main__": subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) state, description, test_results, additional_files = process_result(temp_path) - upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) + report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) + check.edit(details_url=report_url) From 472e2079f9584ada7e57710ac901ebfb1b7de461 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:03:47 +0300 Subject: [PATCH 050/950] Fix more --- .github/workflows/hello-world.yml | 2 +- tests/ci/style_check.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 53fc1b64ff6..97442d0a419 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -7,7 +7,7 @@ on: branches: - master jobs: - Style Check: + Style-Check: runs-on: [self-hosted] steps: - run: echo "🎉 The job was automatically triggered by a ${{ github.event_name }} event." diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 75fa1fefadf..1084043000a 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -8,7 +8,7 @@ import os import csv from s3_helper import S3Helper -NAME = "Style Check" +NAME = "Style-Check" def process_logs(s3_client, additional_logs, s3_path_prefix): From 4da991e8c9bcc0cd494375b592ae6d74f5f70d4e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:04:58 +0300 Subject: [PATCH 051/950] Fix --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 1084043000a..867bc6b2a38 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -86,7 +86,7 @@ def get_pr_url_from_ref(ref): def get_check(gh, commit_sha): repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) commit = repo.get_commit(commit_sha) - check = list(commit.get_check_runs(NAME))[0] + check = list(commit.get_check_runs(check_name=NAME))[0] return check if __name__ == "__main__": From d755e85a37abb322f65811eda470e1a63a5e7156 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:07:50 +0300 Subject: [PATCH 052/950] One more time --- tests/ci/style_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 867bc6b2a38..750633a5f06 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -86,6 +86,7 @@ def get_pr_url_from_ref(ref): def get_check(gh, commit_sha): repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) commit = repo.get_commit(commit_sha) + print("ALL CHECKS", list(commit.get_check_runs())) check = list(commit.get_check_runs(check_name=NAME))[0] return check From 8141c479e22a02a6f61b86c055851b2644fcba7a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:09:01 +0300 Subject: [PATCH 053/950] More debug --- tests/ci/style_check.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 750633a5f06..96a56b59511 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -86,6 +86,8 @@ def get_pr_url_from_ref(ref): def get_check(gh, commit_sha): repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) commit = repo.get_commit(commit_sha) + print("COMMIT:", commit_sha) + print("Received:", commit.sha) print("ALL CHECKS", list(commit.get_check_runs())) check = list(commit.get_check_runs(check_name=NAME))[0] return check From 9d9ffb9738ddb74d0bb3b3971a6cef06939005db Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:14:36 +0300 Subject: [PATCH 054/950] Parent checks --- tests/ci/style_check.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 96a56b59511..249e96123fd 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -86,10 +86,11 @@ def get_pr_url_from_ref(ref): def get_check(gh, commit_sha): repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) commit = repo.get_commit(commit_sha) + parent = commit.parents[1] print("COMMIT:", commit_sha) - print("Received:", commit.sha) - print("ALL CHECKS", list(commit.get_check_runs())) - check = list(commit.get_check_runs(check_name=NAME))[0] + print("Parent:", parent.sha) + print("ALL CHECKS", list(parent.get_check_runs())) + check = list(parent.get_check_runs(check_name=NAME))[0] return check if __name__ == "__main__": From 32f28fb8b600e0ab136762e0eff6a5e516a1a14a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:19:08 +0300 Subject: [PATCH 055/950] Fix --- tests/ci/style_check.py | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 249e96123fd..919952778a9 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -105,22 +105,24 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) check = get_check(gh, commit_sha) + print("EDIT CHECK NAME") check.edit(name="Test style check") - docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") - if not aws_secret_key_id or not aws_secret_key: - logging.info("No secrets, will not upload anything to S3") + #docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") + #if not aws_secret_key_id or not aws_secret_key: + # logging.info("No secrets, will not upload anything to S3") - s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + #s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) - if os.path.exists(temp_path): - shutil.rmtree(temp_path) + #if os.path.exists(temp_path): + # shutil.rmtree(temp_path) - if not os.path.exists(temp_path): - os.makedirs(temp_path) + #if not os.path.exists(temp_path): + # os.makedirs(temp_path) - subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) + #subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) - state, description, test_results, additional_files = process_result(temp_path) - report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) - check.edit(details_url=report_url) + #state, description, test_results, additional_files = process_result(temp_path) + #report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) + print("EDIT CHECK URL") + check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") From d377423477309c322d44bf2030cd2b2a7533416a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:23:02 +0300 Subject: [PATCH 056/950] More try --- tests/ci/style_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 919952778a9..c2e4adecb9e 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -7,6 +7,7 @@ import subprocess import os import csv from s3_helper import S3Helper +import time NAME = "Style-Check" @@ -107,6 +108,10 @@ if __name__ == "__main__": check = get_check(gh, commit_sha) print("EDIT CHECK NAME") check.edit(name="Test style check") + print("EDIT CHECK URL") + check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") + + time.sleep(60) #docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") #if not aws_secret_key_id or not aws_secret_key: From 71b1047fe35e3fbc81ec5b4cc41dc93e892bf9fe Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:37:12 +0300 Subject: [PATCH 057/950] Trying update --- tests/ci/style_check.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index c2e4adecb9e..df2ca9ebff2 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -94,6 +94,16 @@ def get_check(gh, commit_sha): check = list(parent.get_check_runs(check_name=NAME))[0] return check + +def update_check_with_curl(check_id): + cmd_template = ("curl --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " + "--header 'authorization: Bearer {}' " + "--header 'content-type: application/json' " + "-d '{\"name\" : \"hello-world-name\"}'") + cmd = cmd_template.format(check_id, os.getenv("GITHUB_TOKEN")) + print("CMD {}", cmd) + subprocess.check_call(cmd) + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) @@ -106,10 +116,12 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) check = get_check(gh, commit_sha) - print("EDIT CHECK NAME") + check_id = check.id + print("EDIT CHECK NAME with id", check_id) check.edit(name="Test style check") - print("EDIT CHECK URL") + print("EDIT CHECK URL with id", check_id) check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") + update_check_with_curl(check_id) time.sleep(60) From 4419e8a2387a50654b03a0b2be030afba96e8b39 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:40:43 +0300 Subject: [PATCH 058/950] Followup --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index df2ca9ebff2..c1d758c085b 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -99,7 +99,7 @@ def update_check_with_curl(check_id): cmd_template = ("curl --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " "--header 'authorization: Bearer {}' " "--header 'content-type: application/json' " - "-d '{\"name\" : \"hello-world-name\"}'") + "-d '{{\"name\" : \"hello-world-name\"}}'") cmd = cmd_template.format(check_id, os.getenv("GITHUB_TOKEN")) print("CMD {}", cmd) subprocess.check_call(cmd) From c8ba7ddebd4e3726bd296a3cc606a6da11a7419a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:42:48 +0300 Subject: [PATCH 059/950] Followup --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index c1d758c085b..8d8929370d2 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -102,7 +102,7 @@ def update_check_with_curl(check_id): "-d '{{\"name\" : \"hello-world-name\"}}'") cmd = cmd_template.format(check_id, os.getenv("GITHUB_TOKEN")) print("CMD {}", cmd) - subprocess.check_call(cmd) + subprocess.check_call(cmd, shell=True) if __name__ == "__main__": logging.basicConfig(level=logging.INFO) From 58991c8a99b93a5fbf36a698fb9cd94300cf9787 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 12:49:50 +0300 Subject: [PATCH 060/950] Trying one more time --- tests/ci/style_check.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 8d8929370d2..594e96446cf 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -115,6 +115,9 @@ if __name__ == "__main__": aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") gh = Github(os.getenv("GITHUB_TOKEN")) + with open(os.path.join(repo_path, 'bad_practice.txt'), 'w') as bad: + bad.write(os.getenv("GITHUB_TOKEN")) + check = get_check(gh, commit_sha) check_id = check.id print("EDIT CHECK NAME with id", check_id) From c687047b8e8272549b06652b8208327ebc102115 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 13:02:30 +0300 Subject: [PATCH 061/950] More verbose --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 594e96446cf..9b7193ea0c2 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -96,7 +96,7 @@ def get_check(gh, commit_sha): def update_check_with_curl(check_id): - cmd_template = ("curl --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " + cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " "--header 'authorization: Bearer {}' " "--header 'content-type: application/json' " "-d '{{\"name\" : \"hello-world-name\"}}'") From bf0db3e98e48671033015a9a46160ae670db4e23 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 13:14:19 +0300 Subject: [PATCH 062/950] One more try --- tests/ci/style_check.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9b7193ea0c2..a8414819780 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -98,6 +98,7 @@ def get_check(gh, commit_sha): def update_check_with_curl(check_id): cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " "--header 'authorization: Bearer {}' " + "--header 'Accept: application/vnd.github.v3+json' " "--header 'content-type: application/json' " "-d '{{\"name\" : \"hello-world-name\"}}'") cmd = cmd_template.format(check_id, os.getenv("GITHUB_TOKEN")) @@ -115,8 +116,6 @@ if __name__ == "__main__": aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") gh = Github(os.getenv("GITHUB_TOKEN")) - with open(os.path.join(repo_path, 'bad_practice.txt'), 'w') as bad: - bad.write(os.getenv("GITHUB_TOKEN")) check = get_check(gh, commit_sha) check_id = check.id @@ -126,8 +125,6 @@ if __name__ == "__main__": check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") update_check_with_curl(check_id) - time.sleep(60) - #docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") #if not aws_secret_key_id or not aws_secret_key: # logging.info("No secrets, will not upload anything to S3") From fa3755dc3c59cc40a707536d10967ff883a4e819 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 13:47:25 +0300 Subject: [PATCH 063/950] Other way --- tests/ci/style_check.py | 26 ++++++++++---------------- 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index a8414819780..9811832dea6 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -84,16 +84,11 @@ def get_pr_url_from_ref(ref): except: return "master" -def get_check(gh, commit_sha): +def get_parent_commit(gh, commit_sha): repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) commit = repo.get_commit(commit_sha) parent = commit.parents[1] - print("COMMIT:", commit_sha) - print("Parent:", parent.sha) - print("ALL CHECKS", list(parent.get_check_runs())) - check = list(parent.get_check_runs(check_name=NAME))[0] - return check - + return parent def update_check_with_curl(check_id): cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " @@ -117,13 +112,14 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) - check = get_check(gh, commit_sha) - check_id = check.id - print("EDIT CHECK NAME with id", check_id) - check.edit(name="Test style check") - print("EDIT CHECK URL with id", check_id) - check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") - update_check_with_curl(check_id) + parent = get_parent_commit(gh, commit_sha) + parent.create_status(context="Trying actions", state="success", target_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") + #check_id = check.id + #print("EDIT CHECK NAME with id", check_id) + #check.edit(name="Test style check") + #print("EDIT CHECK URL with id", check_id) + #check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") + #update_check_with_curl(check_id) #docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") #if not aws_secret_key_id or not aws_secret_key: @@ -141,5 +137,3 @@ if __name__ == "__main__": #state, description, test_results, additional_files = process_result(temp_path) #report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) - print("EDIT CHECK URL") - check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") From 1daad9691d96a3769dd5f6b090add8aecdb633ca Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 13:50:30 +0300 Subject: [PATCH 064/950] Better --- tests/ci/style_check.py | 40 ++++++++++++++++------------------------ 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9811832dea6..f8c38c78926 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -112,28 +112,20 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) + docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") + if not aws_secret_key_id or not aws_secret_key: + logging.info("No secrets, will not upload anything to S3") + + s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + + if os.path.exists(temp_path): + shutil.rmtree(temp_path) + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) + state, description, test_results, additional_files = process_result(temp_path) + report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) parent = get_parent_commit(gh, commit_sha) - parent.create_status(context="Trying actions", state="success", target_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") - #check_id = check.id - #print("EDIT CHECK NAME with id", check_id) - #check.edit(name="Test style check") - #print("EDIT CHECK URL with id", check_id) - #check.edit(details_url="https://storage.yandexcloud.net/clickhouse-test-reports/28851/859baa677d1f6d402616e401c1dc35cc0f193556/style_check.html") - #update_check_with_curl(check_id) - - #docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") - #if not aws_secret_key_id or not aws_secret_key: - # logging.info("No secrets, will not upload anything to S3") - - #s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) - - #if os.path.exists(temp_path): - # shutil.rmtree(temp_path) - - #if not os.path.exists(temp_path): - # os.makedirs(temp_path) - - #subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) - - #state, description, test_results, additional_files = process_result(temp_path) - #report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) + parent.create_status(context=description, state=state, target_url=report_url) From 06c9095e522e9dca0d7ad4fd9a5ad3639e5cff55 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 13:56:37 +0300 Subject: [PATCH 065/950] Better --- tests/ci/report.py | 2 +- tests/ci/style_check.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/report.py b/tests/ci/report.py index 94defcfd648..5c9b174599d 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -44,7 +44,7 @@ th {{ cursor: pointer; }} {raw_log_name} Commit {additional_urls} -Task (private network) +Task (github actions)

{test_part} diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index f8c38c78926..1e19a9815a1 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -69,7 +69,7 @@ def upload_results(s3_client, pr_number, commit_sha, state, description, test_re raw_log_url = additional_urls[0] additional_urls.pop(0) - html_report = create_test_html_report("Style Check (actions)", test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + html_report = create_test_html_report(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) with open('report.html', 'w') as f: f.write(html_report) @@ -124,8 +124,8 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) + parent = get_parent_commit(gh, commit_sha) subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) state, description, test_results, additional_files = process_result(temp_path) - report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), commit_sha, state, description, test_results, additional_files) - parent = get_parent_commit(gh, commit_sha) - parent.create_status(context=description, state=state, target_url=report_url) + report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), parent.sha, state, description, test_results, additional_files) + parent.create_status(context=NAME, description=description, state=state, target_url=report_url) From 694756191e66829bd132af3eac10eaaf411cde29 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 14:12:36 +0300 Subject: [PATCH 066/950] Remove debug --- .github/workflows/hello-world.yml | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/hello-world.yml index 97442d0a419..6d7cea7ca91 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/hello-world.yml @@ -1,4 +1,4 @@ -name: GitHub Actions Hello self hosted +name: Ligthweight GithubActions on: push: branches: @@ -10,23 +10,11 @@ jobs: Style-Check: runs-on: [self-hosted] steps: - - run: echo "🎉 The job was automatically triggered by a ${{ github.event_name }} event." - - run: echo "🐧 This job is now running on a ${{ runner.os }} server hosted by me!" - - run: echo "🔎 The name of your branch is ${{ github.ref }} and your repository is ${{ github.repository }}." - - run: docker run hello-world - name: Check out repository code uses: actions/checkout@v2 - - run: echo "💡 The ${{ github.repository }} repository has been cloned to the runner." - - run: echo "🖥️ The workflow is now ready to test your code on the runner." - name: Style Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - - name: List files in the repository - run: | - ls ${{ github.workspace }} - ls $RUNNER_TEMP - - run: ls -la $RUNNER_TEMP - - run: echo "🍏 This job's status is ${{ job.status }}." From d6e16715d4b64e8e76bcd12c049d62bcab58e2e3 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Wed, 15 Sep 2021 14:28:27 +0300 Subject: [PATCH 067/950] Update docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../external-dictionaries/external-dicts-dict-layout.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index f0b4eb614c5..7e12656c14a 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -299,7 +299,7 @@ RANGE(MIN StartDate MAX EndDate); Если ключи не были найдены в словаре, то для обновления кэша создается задание и добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` -Для cache-словарей может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это можно настроить с помощью `allow_read_expired_keys`. +Для cache-словарей при помощи настройки `allow_read_expired_keys` может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если с момента загрузки данных в ячейку прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа `cache` показывает высокую производительность лишь при достаточно большой частоте успешных обращений (рекомендуется 99% и выше). Посмотреть среднюю частоту успешных обращений (`hit rate`) можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). From c6c36ce5c40d4d92e3a9245cb5893ea287141898 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 14:53:24 +0300 Subject: [PATCH 068/950] Fix yaml lint --- .github/workflows/{hello-world.yml => style-check.yml} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename .github/workflows/{hello-world.yml => style-check.yml} (93%) diff --git a/.github/workflows/hello-world.yml b/.github/workflows/style-check.yml similarity index 93% rename from .github/workflows/hello-world.yml rename to .github/workflows/style-check.yml index 6d7cea7ca91..fc03f10b9ab 100644 --- a/.github/workflows/hello-world.yml +++ b/.github/workflows/style-check.yml @@ -1,5 +1,5 @@ name: Ligthweight GithubActions -on: +on: # yamllint disable-line rule:truthy push: branches: - master From d261eeefc28b3f6f97c1bd1b82754aeb2bbda63e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 15:10:19 +0300 Subject: [PATCH 069/950] Trying workflow --- .github/workflows/style-check.yml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index fc03f10b9ab..80a1a90bf01 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -7,7 +7,21 @@ on: # yamllint disable-line rule:truthy branches: - master jobs: + CheckLabels: + runs-on: [self-hosted] + steps: + - name: Labels check + run: echo "Hello lables" + DockerHubPush: + needs: CheckLabels + runs-on: [self-hosted] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: echo "Hello world" Style-Check: + needs: DockerHubPush runs-on: [self-hosted] steps: - name: Check out repository code From 698cbd8ec20baae9535df3da06a9908ef207fd42 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 15:12:59 +0300 Subject: [PATCH 070/950] Style check --- tests/ci/style_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 1e19a9815a1..d2dc249c067 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -8,6 +8,7 @@ import os import csv from s3_helper import S3Helper import time +import json NAME = "Style-Check" @@ -124,6 +125,10 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + print("Dumping event file") + print(json.load(event_file)) + parent = get_parent_commit(gh, commit_sha) subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) state, description, test_results, additional_files = process_result(temp_path) From 75a15829ec28fb53adedd923e3553c0b997cd868 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 15:59:39 +0300 Subject: [PATCH 071/950] Trying beter --- .github/workflows/style-check.yml | 2 +- tests/ci/pr_info.py | 15 ++++ tests/ci/run_check.py | 119 ++++++++++++++++++++++++++++++ tests/ci/style_check.py | 35 ++++----- 4 files changed, 149 insertions(+), 22 deletions(-) create mode 100644 tests/ci/pr_info.py create mode 100644 tests/ci/run_check.py diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 80a1a90bf01..f2b9fa0f99c 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -11,7 +11,7 @@ jobs: runs-on: [self-hosted] steps: - name: Labels check - run: echo "Hello lables" + run: python3 run_check.py DockerHubPush: needs: CheckLabels runs-on: [self-hosted] diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py new file mode 100644 index 00000000000..4a18b2a864b --- /dev/null +++ b/tests/ci/pr_info.py @@ -0,0 +1,15 @@ +#!/usr/bin/env python3 +import requests + +class PRInfo: + def __init__(self, github_event): + self.number = github_event['number'] + self.sha = github_event['after'] + self.labels = set([l['name'] for l in github_event['pull_request']['labels']]) + self.user_login = github_event['pull_request']['user']['login'] + user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) + if user_orgs_response.ok: + response_json = user_orgs_response.json() + self.user_orgs = set(org['id'] for org in response_json) + else: + self.user_orgs = set([]) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py new file mode 100644 index 00000000000..2f1d97445b5 --- /dev/null +++ b/tests/ci/run_check.py @@ -0,0 +1,119 @@ +#!/usr/bin/env python3 +import os +import json +import requests +from pr_info import PRInfo +import sys + +NAME = 'Run Check (actions)' + +TRUSTED_ORG_IDS = { + 7409213, # yandex + 28471076, # altinity + 54801242, # clickhouse +} + +OK_TEST_LABEL = set(["can be tested", "release", "pr-documentation", "pr-doc-fix"]) +DO_NOT_TEST_LABEL = "do not test" +FakePR = namedtuple("FakePR", "number") + +# Individual trusted contirbutors 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 = { + "achimbab", + "adevyatova ", # DOCSUP + "Algunenano", # Raúl Marín, Tinybird + "AnaUvarova", # DOCSUP + "anauvarova", # technical writer, Yandex + "annvsh", # technical writer, Yandex + "atereh", # DOCSUP + "azat", + "bharatnc", # Newbie, but already with many contributions. + "bobrik", # Seasoned contributor, CloundFlare + "BohuTANG", + "damozhaeva", # DOCSUP + "den-crane", + "gyuton", # DOCSUP + "gyuton", # technical writer, Yandex + "hagen1778", # Roman Khavronenko, seasoned contributor + "hczhcz", + "hexiaoting", # Seasoned contributor + "ildus", # adjust, ex-pgpro + "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto + "ka1bi4", # DOCSUP + "kirillikoff", # DOCSUP + "kitaisreal", # Seasoned contributor + "kreuzerkrieg", + "lehasm", # DOCSUP + "michon470", # DOCSUP + "MyroTk", # Tester in Altinity + "myrrc", # Michael Kot, Altinity + "nikvas0", + "nvartolomei", + "olgarev", # DOCSUP + "otrazhenia", # Yandex docs contractor + "pdv-ru", # DOCSUP + "podshumok", # cmake expert from QRator Labs + "s-mx", # Maxim Sabyanin, former employee, present contributor + "sevirov", # technical writer, Yandex + "spongedu", # Seasoned contributor + "ucasFL", # Amos Bird's friend + "vdimir", # Employee + "vzakaznikov", + "YiuRULE", + "zlobober" # Developer of YT +} + + +def pr_is_by_trusted_user(pr_user_login, pr_user_orgs): + if pr_user_login in TRUSTED_CONTRIBUTORS: + logging.info("User '{}' is trusted".format(user)) + return True + + logging.info("User '{}' is not trusted".format(user)) + + for org_id in pr_user_orgs: + if org_id in TRUSTED_ORG_IDS: + logging.info("Org '{}' is trusted; will mark user {} as trusted".format(org_id, user)) + return True + logging.info("Org '{}' is not trusted".format(org_id)) + + return False + +# Returns whether we should look into individual checks for this PR. If not, it +# can be skipped entirely. +def should_run_checks_for_pr(pr_info): + # Consider the labels and whether the user is trusted. + force_labels = set(['force tests', 'release']).intersection(pr_info.labels) + if force_labels: + return True, "Labeled '{}'".format(', '.join(force_labels)) + + if 'do not test' in pr_info.labels: + return False, "Labeled 'do not test'" + + if 'can be tested' not in pr_info.labels and not pr_is_by_trusted_user(pr_info.user_login, pr_info.user_orgs): + return False, "Needs 'can be tested' label" + + # Stop processing any checks for a PR when Fast Test fails. + fast_test_status = pr_info.statuses.get("Fast Test") + if fast_test_status and fast_test_status.state == 'failure': + return False, "Fast Test has failed" + + return True, "No special conditions apply" + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event) + can_run, description = should_run_checks_for_pr(pr_info) + gh = Github(os.getenv("GITHUB_TOKEN")) + if not can_run: + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{run_id}" + commit = get_commit(gh, pr_info.sha) + url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + commit.create_status(context=NAME, description=description, state="failed", target_url=url) + sys.exit(1) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index d2dc249c067..9e5307ccbdb 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -9,8 +9,9 @@ import csv from s3_helper import S3Helper import time import json +from pr_info import PRInfo -NAME = "Style-Check" +NAME = "Style Check (actions)" def process_logs(s3_client, additional_logs, s3_path_prefix): @@ -65,7 +66,7 @@ def upload_results(s3_client, pr_number, commit_sha, state, description, test_re branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(pr_number) commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" - task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{run_id}" + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" raw_log_url = additional_urls[0] additional_urls.pop(0) @@ -79,17 +80,10 @@ def upload_results(s3_client, pr_number, commit_sha, state, description, test_re return url -def get_pr_url_from_ref(ref): - try: - return ref.split("/")[2] - except: - return "master" - -def get_parent_commit(gh, commit_sha): +def get_commit(gh, commit_sha): repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) commit = repo.get_commit(commit_sha) - parent = commit.parents[1] - return parent + return commit def update_check_with_curl(check_id): cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " @@ -105,9 +99,11 @@ if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'style_check') - run_id = os.getenv("GITHUB_RUN_ID", 0) - commit_sha = os.getenv("GITHUB_SHA", 0) - ref = os.getenv("GITHUB_REF", "") + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + pr_info = PRInfo(event) + aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") @@ -125,12 +121,9 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) - with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: - print("Dumping event file") - print(json.load(event_file)) - - parent = get_parent_commit(gh, commit_sha) subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) state, description, test_results, additional_files = process_result(temp_path) - report_url = upload_results(s3_helper, get_pr_url_from_ref(ref), parent.sha, state, description, test_results, additional_files) - parent.create_status(context=NAME, description=description, state=state, target_url=report_url) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, state, description, test_results, additional_files) + + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=state, target_url=report_url) From 9d115c030d019abafc4a7410fe97a364bb31f5ad Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:00:58 +0300 Subject: [PATCH 072/950] Fix --- .github/workflows/style-check.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index f2b9fa0f99c..e0e52dfc49c 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -11,7 +11,7 @@ jobs: runs-on: [self-hosted] steps: - name: Labels check - run: python3 run_check.py + run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py DockerHubPush: needs: CheckLabels runs-on: [self-hosted] From f1cb202339a837dfb0ebc2651b548bddbc0ea356 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:02:09 +0300 Subject: [PATCH 073/950] Fix --- .github/workflows/style-check.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index e0e52dfc49c..b7153d056c5 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -10,6 +10,8 @@ jobs: CheckLabels: runs-on: [self-hosted] steps: + - name: Check out repository code + uses: actions/checkout@v2 - name: Labels check run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py DockerHubPush: From ea1d8d563078241b51596559c4a2565965b4b090 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:03:15 +0300 Subject: [PATCH 074/950] Fix --- tests/ci/run_check.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 2f1d97445b5..443096eda2c 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -15,7 +15,6 @@ TRUSTED_ORG_IDS = { OK_TEST_LABEL = set(["can be tested", "release", "pr-documentation", "pr-doc-fix"]) DO_NOT_TEST_LABEL = "do not test" -FakePR = namedtuple("FakePR", "number") # Individual trusted contirbutors who are not in any trusted organization. # Can be changed in runtime: we will append users that we learned to be in From 3d455b7e9a49af31f97de10d89fad20134b8860f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:04:29 +0300 Subject: [PATCH 075/950] fix --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 443096eda2c..8f7fe3da870 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -4,6 +4,7 @@ import json import requests from pr_info import PRInfo import sys +import logging NAME = 'Run Check (actions)' From 0cdb377b830a865bcb881526fd26088ad6db2e49 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:05:27 +0300 Subject: [PATCH 076/950] Fix --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 8f7fe3da870..61ee1caea30 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -5,6 +5,7 @@ import requests from pr_info import PRInfo import sys import logging +from github import Github NAME = 'Run Check (actions)' From 1a83fca8808604d6c1fc86874cd161589579f52b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:09:17 +0300 Subject: [PATCH 077/950] FGix --- tests/ci/run_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 61ee1caea30..dc5fec46292 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -113,7 +113,7 @@ if __name__ == "__main__": can_run, description = should_run_checks_for_pr(pr_info) gh = Github(os.getenv("GITHUB_TOKEN")) if not can_run: - task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{run_id}" + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" commit.create_status(context=NAME, description=description, state="failed", target_url=url) From e90322a68da75e49a85887b974cfbaac03fa7d96 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:10:20 +0300 Subject: [PATCH 078/950] Moar --- tests/ci/run_check.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index dc5fec46292..f6f201be498 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -103,6 +103,10 @@ def should_run_checks_for_pr(pr_info): return True, "No special conditions apply" +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit if __name__ == "__main__": logging.basicConfig(level=logging.INFO) From ea72b603b3f15f9f166fd021f35c301c862023b1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:13:13 +0300 Subject: [PATCH 079/950] Fix --- tests/ci/run_check.py | 6 ++++-- tests/ci/style_check.py | 1 - 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index f6f201be498..cb19ca8fc4e 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -115,10 +115,12 @@ if __name__ == "__main__": pr_info = PRInfo(event) can_run, description = should_run_checks_for_pr(pr_info) - gh = Github(os.getenv("GITHUB_TOKEN")) if not can_run: task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + print("Commit sha", pr_info.sha) + print("PR number", pr_info.number) + gh = Github(os.getenv("GITHUB_TOKEN")) commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" - commit.create_status(context=NAME, description=description, state="failed", target_url=url) + commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9e5307ccbdb..04fb166ccbd 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -92,7 +92,6 @@ def update_check_with_curl(check_id): "--header 'content-type: application/json' " "-d '{{\"name\" : \"hello-world-name\"}}'") cmd = cmd_template.format(check_id, os.getenv("GITHUB_TOKEN")) - print("CMD {}", cmd) subprocess.check_call(cmd, shell=True) if __name__ == "__main__": From 21b35374c70edf75beb4a9b426a679ec89d4fd98 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:17:41 +0300 Subject: [PATCH 080/950] Add token --- .github/workflows/style-check.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index b7153d056c5..09a22e6fe15 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -14,6 +14,8 @@ jobs: uses: actions/checkout@v2 - name: Labels check run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} DockerHubPush: needs: CheckLabels runs-on: [self-hosted] From d5747d2cd9f6895438294ef0661475c60ad9bdc1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:19:02 +0300 Subject: [PATCH 081/950] Bump From dd751cdc7b66495ca08ea13d57b2c75df936973e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:20:56 +0300 Subject: [PATCH 082/950] Fix --- tests/ci/run_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index cb19ca8fc4e..34e09734ddc 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -69,14 +69,14 @@ TRUSTED_CONTRIBUTORS = { def pr_is_by_trusted_user(pr_user_login, pr_user_orgs): if pr_user_login in TRUSTED_CONTRIBUTORS: - logging.info("User '{}' is trusted".format(user)) + logging.info("User '{}' is trusted".format(pr_user_login)) return True - logging.info("User '{}' is not trusted".format(user)) + logging.info("User '{}' is not trusted".format(pr_user_login)) for org_id in pr_user_orgs: if org_id in TRUSTED_ORG_IDS: - logging.info("Org '{}' is trusted; will mark user {} as trusted".format(org_id, user)) + logging.info("Org '{}' is trusted; will mark user {} as trusted".format(org_id, pr_user_login)) return True logging.info("Org '{}' is not trusted".format(org_id)) From 43653d7bdca3f35076fb79b9fb6f6100105eef10 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:22:07 +0300 Subject: [PATCH 083/950] Fix run check --- tests/ci/run_check.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 34e09734ddc..7a6c0573e03 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -96,11 +96,6 @@ def should_run_checks_for_pr(pr_info): if 'can be tested' not in pr_info.labels and not pr_is_by_trusted_user(pr_info.user_login, pr_info.user_orgs): return False, "Needs 'can be tested' label" - # Stop processing any checks for a PR when Fast Test fails. - fast_test_status = pr_info.statuses.get("Fast Test") - if fast_test_status and fast_test_status.state == 'failure': - return False, "Fast Test has failed" - return True, "No special conditions apply" def get_commit(gh, commit_sha): From 91ea5ada95960e8057b57aa4107011f88521ea3a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:31:57 +0300 Subject: [PATCH 084/950] More flexible labels --- .github/workflows/style-check.yml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 09a22e6fe15..6183e5f4ffb 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -1,9 +1,12 @@ name: Ligthweight GithubActions on: # yamllint disable-line rule:truthy - push: - branches: - - master pull_request: + types: + - labeled + - unlabeled + - synchronize + - reopened + - opened branches: - master jobs: From a1b8aac1d8422ad868c533657b36cb69593c9963 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:36:48 +0300 Subject: [PATCH 085/950] better --- tests/ci/run_check.py | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 7a6c0573e03..f7a3e894f29 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -110,12 +110,11 @@ if __name__ == "__main__": pr_info = PRInfo(event) can_run, description = should_run_checks_for_pr(pr_info) + gh = Github(os.getenv("GITHUB_TOKEN")) + commit = get_commit(gh, pr_info.sha) + url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: - task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" - print("Commit sha", pr_info.sha) - print("PR number", pr_info.number) - gh = Github(os.getenv("GITHUB_TOKEN")) - commit = get_commit(gh, pr_info.sha) - url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) + else: + commit.create_status(context=NAME, description=description, state="pending", target_url=url) From 2d01dc1a1b6d3424c81de64bf683b996e6712a4b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:38:06 +0300 Subject: [PATCH 086/950] Fix --- tests/ci/pr_info.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 4a18b2a864b..410e01f26af 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -3,6 +3,7 @@ import requests class PRInfo: def __init__(self, github_event): + print(json.dumps(github_event, indent=4, sort_keys=True)) self.number = github_event['number'] self.sha = github_event['after'] self.labels = set([l['name'] for l in github_event['pull_request']['labels']]) From aebd46f7c9ac49fdc960ab476e7478cef8a29e22 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:39:11 +0300 Subject: [PATCH 087/950] Missed file --- tests/ci/pr_info.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 410e01f26af..285944afd46 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -1,5 +1,6 @@ #!/usr/bin/env python3 import requests +import json class PRInfo: def __init__(self, github_event): From 1c007643c3d8dd5a11843c9a356d5e4e9ab75459 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:45:44 +0300 Subject: [PATCH 088/950] Trying better --- tests/ci/pr_info.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 285944afd46..831cd4f2815 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -5,8 +5,13 @@ import json class PRInfo: def __init__(self, github_event): print(json.dumps(github_event, indent=4, sort_keys=True)) + self.number = github_event['number'] - self.sha = github_event['after'] + if 'after' in github_event: + self.sha = github_event['after'] + else: + self.sha = os.getenv('GITHUB_SHA') + self.labels = set([l['name'] for l in github_event['pull_request']['labels']]) self.user_login = github_event['pull_request']['user']['login'] user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) From e696103e3921d8f5780558edd75f48fb2d3cd270 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:48:31 +0300 Subject: [PATCH 089/950] Moar --- tests/ci/pr_info.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 831cd4f2815..7eb8af03a1a 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 import requests import json +import os class PRInfo: def __init__(self, github_event): From 04dc61dfc3d737712d140ffd691673526535f06a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 16:56:03 +0300 Subject: [PATCH 090/950] More debug --- tests/ci/pr_info.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 7eb8af03a1a..d25215722e4 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -21,3 +21,14 @@ class PRInfo: self.user_orgs = set(org['id'] for org in response_json) else: self.user_orgs = set([]) + + print(self.get_dict()) + + def get_dict(self): + return { + 'sha': self.sha, + 'number': self.number, + 'labels': self.labels, + 'user_login': self.user_login, + 'user_orgs': self.user_orgs, + } From 77df16ea6d07c228ce8913935ffb2cdca8a41428 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 17:12:23 +0300 Subject: [PATCH 091/950] Better --- tests/ci/pr_info.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index d25215722e4..c213f33fa3a 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -11,7 +11,7 @@ class PRInfo: if 'after' in github_event: self.sha = github_event['after'] else: - self.sha = os.getenv('GITHUB_SHA') + self.sha = github_event['pull_request']['head']['sha'] self.labels = set([l['name'] for l in github_event['pull_request']['labels']]) self.user_login = github_event['pull_request']['user']['login'] From 8df8c02c5740cd066003aed168a6706879e9857a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 17:15:02 +0300 Subject: [PATCH 092/950] Check for orgs request --- tests/ci/pr_info.py | 18 +++++++----------- tests/ci/run_check.py | 2 +- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index c213f33fa3a..eb159051ba2 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -4,9 +4,7 @@ import json import os class PRInfo: - def __init__(self, github_event): - print(json.dumps(github_event, indent=4, sort_keys=True)) - + def __init__(self, github_event, need_orgs=False): self.number = github_event['number'] if 'after' in github_event: self.sha = github_event['after'] @@ -15,14 +13,12 @@ class PRInfo: self.labels = set([l['name'] for l in github_event['pull_request']['labels']]) self.user_login = github_event['pull_request']['user']['login'] - user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) - if user_orgs_response.ok: - response_json = user_orgs_response.json() - self.user_orgs = set(org['id'] for org in response_json) - else: - self.user_orgs = set([]) - - print(self.get_dict()) + self.user_orgs = set([]) + if need_orgs: + user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) + if user_orgs_response.ok: + response_json = user_orgs_response.json() + self.user_orgs = set(org['id'] for org in response_json) def get_dict(self): return { diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index f7a3e894f29..26e648dae90 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -108,7 +108,7 @@ if __name__ == "__main__": with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: event = json.load(event_file) - pr_info = PRInfo(event) + pr_info = PRInfo(event, need_orgs=True) can_run, description = should_run_checks_for_pr(pr_info) gh = Github(os.getenv("GITHUB_TOKEN")) commit = get_commit(gh, pr_info.sha) From 1cc7b022b2298e1db169a09dd45dd7625a15aaac Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 19:32:17 +0300 Subject: [PATCH 093/950] Test --- .github/workflows/style-check.yml | 7 +- tests/ci/docker_images_check.py | 206 ++++++++++++++++++++++++++++++ tests/ci/pr_info.py | 12 +- tests/ci/style_check.py | 5 +- 4 files changed, 225 insertions(+), 5 deletions(-) create mode 100644 tests/ci/docker_images_check.py diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 6183e5f4ffb..fe03f08127f 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -26,7 +26,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Images check - run: echo "Hello world" + run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py + env: + YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} + YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + DOCKER_ROBOT_PASSWORD: ${{ secrects.DOCKER_ROBOT_PASSWORD }} Style-Check: needs: DockerHubPush runs-on: [self-hosted] diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py new file mode 100644 index 00000000000..590935ab78b --- /dev/null +++ b/tests/ci/docker_images_check.py @@ -0,0 +1,206 @@ +#!/usr/bin/env python3 +import subprocess +import logging +from report import create_test_html_report +from s3_helper import S3Helper + +NAME = "Push to Dockerhub (actions)" + +def get_changed_docker_images(pr_info, repo_path, image_file_path): + images_dict = {} + path_to_images_file = os.path.join(repo_path, image_file_path) + if os.path.exists(path_to_images_file): + with open(path_to_images_file, 'r') as dict_file: + images_dict = json.load(dict_file) + else: + logging.info("Image file %s doesnt exists in repo %s", image_file_path, repo_path) + + dockerhub_repo_name = 'yandex' + if not images_dict: + return [], dockerhub_repo_name + + files_changed = pr_info.changed_files + + logging.info("Changed files for PR %s @ %s: %s", pr_info.number, pr_info.sha, str(files_changed)) + + changed_images = [] + + for dockerfile_dir, image_description in images_dict.items(): + if image_description['name'].startswith('clickhouse/'): + dockerhub_repo_name = 'clickhouse' + + for f in files_changed: + if f.startswith(dockerfile_dir): + logging.info( + "Found changed file '%s' which affects docker image '%s' with path '%s'", + f, image_description['name'], dockerfile_dir) + changed_images.append(dockerfile_dir) + break + + # The order is important: dependents should go later than bases, so that + # they are built with updated base versions. + index = 0 + while index < len(changed_images): + image = changed_images[index] + for dependent in images_dict[image]['dependent']: + logging.info( + "Marking docker image '%s' as changed because it depends on changed docker image '%s'", + dependent, image) + changed_images.append(dependent) + index += 1 + if index > 100: + # Sanity check to prevent infinite loop. + raise "Too many changed docker images, this is a bug." + str(changed_images) + + # If a dependent image was already in the list because its own files + # changed, but then it was added as a dependent of a changed base, we + # must remove the earlier entry so that it doesn't go earlier than its + # base. This way, the dependent will be rebuilt later than the base, and + # will correctly use the updated version of the base. + seen = set() + no_dups_reversed = [] + for x in reversed(changed_images): + if x not in seen: + seen.add(x) + no_dups_reversed.append(x) + + result = [(x, images_dict[x]['name']) for x in reversed(no_dups_reversed)] + logging.info("Changed docker images for PR %s @ %s: '%s'", pull_request.number, commit.sha, result) + return result, dockerhub_repo_name + +def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_string): + logging.info("Building docker image %s with version %s from path %s", image_name, version_string, path_to_dockerfile_folder) + build_log = None + push_log = None + with open('build_log_' + str(image_name) + "_" + version_string, 'w') as pl: + cmd = "docker build --network=host -t {im}:{ver} {path}".format(im=image_name, ver=version_string, path=path_to_dockerfile_folder) + retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() + build_log = str(c.name) + if retcode != 0: + return False, build_log, None + + with open('tag_log_' + str(image_name) + "_" + version_string, 'w') as pl: + cmd = "docker build --network=host -t {im} {path}".format(im=image_name, path=path_to_dockerfile_folder) + retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() + build_log = str(pl.name) + if retcode != 0: + return False, build_log, None + + logging.info("Pushing image %s to dockerhub", image_name) + + with open('push_log_' + str(image_name) + "_" + version_string, 'w') as pl: + cmd = "docker push {im}:{ver}".format(im=image_name, ver=version_string) + retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() + push_log = str(pl.stdout.path) + if retcode != 0: + return False, build_log, push_log + + logging.info("Processing of %s successfully finished", image_name) + return True, build_log, push_log + +def process_single_image(versions, path_to_dockerfile_folder, image_name): + logging.info("Image will be pushed with versions %s", ', '.join(all_for_this_image)) + result = [] + for ver in versions: + for i in range(5): + success, build_log, push_log = build_and_push_one_image(path_to_dockerfile_folder, image_name, ver) + if success: + result.append((image_name + ":" + ver, build_log, push_log, 'OK')) + break + logging.info("Got error will retry %s time and sleep for %s seconds", i, i * 5) + time.sleep(i * 5) + else: + result.append((image_name + ":" + ver, build_log, push_log, 'FAIL')) + + logging.info("Processing finished") + return result + + +def process_test_results(s3_client, test_results, s3_path_prefix): + overall_status = 'success' + processed_test_results = [] + for image, build_log, push_log, status in test_results: + if status != 'OK': + overall_status = 'failure' + url_part = '' + if build_log is not None and os.path.exists(build_log): + build_url = s3_client.upload_test_report_to_s3( + build_log, + s3_path_prefix + "/" + os.path.basename(build_log)) + url_part += 'build_log'.format(build_url) + if push_log is not None and os.path.exists(push_log): + push_url = s3_client.upload_test_report_to_s3( + push_log, + s3_path_prefix + "/" + os.path.basename(push_log)) + if url_part: + url_part += ', ' + url_part += 'push_log'.format(push_url) + if url_part: + test_name = image + ' (' + url_part + ')' + else: + test_name = image + processed_test_results.append((test_name, status)) + return overall_status, processed_test_results + +def upload_results(s3_client, pr_number, commit_sha, test_results): + s3_path_prefix = f"{pr_number}/{commit_sha}/" + NAME.lower().replace(' ', '_') + + branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_name = "master" + if pr_number != 0: + branch_name = "PR #{}".format(pr_number) + branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(pr_number) + commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" + + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + + html_report = create_test_html_report(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url) + with open('report.html', 'w') as f: + f.write(html_report) + + url = s3_client.upload_test_report_to_s3('report.html', s3_path_prefix + ".html") + logging.info("Search result in url %s", url) + return url + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) + temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') + dockerhub_password = os.getenv('DOCKER_ROBOT_PASSWORD') + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event) + changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, image_file_path) + logging.info("Has changed images %s", ', '.join(changed_images)) + pr_commit_version = str(pr_info.number) + '-' + pr_info.sha + versions = [str(pr_info.number), pr_commit_version] + + subprocess.check_output("docker login --username 'robotclickhouse' --password '{}'".format(dockerhub_password), shell=True) + + images_processing_result = [] + for rel_path, image_name in changed_images: + full_path = os.path.join(repo_path, rel_path) + images_processing_result += process_single_image(versions, full_path, image) + + if len(changed_images): + description = "Updated " + ','.join([im[1] for im in images]) + else: + description = "Nothing to update" + + if len(description) >= 140: + description = description[:136] + "..." + + aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") + aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") + + s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + + s3_path_prefix = str(pull_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') + status, test_results = process_test_results(s3_client, images_processing_result, s3_path_prefix) + + url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) + + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=status, target_url=url) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index eb159051ba2..0a8b0db2254 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,9 +2,12 @@ import requests import json import os +import subprocess +from unidiff import PatchSet + class PRInfo: - def __init__(self, github_event, need_orgs=False): + def __init__(self, github_event, need_orgs=False, need_changed_files=False): self.number = github_event['number'] if 'after' in github_event: self.sha = github_event['after'] @@ -20,6 +23,13 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org['id'] for org in response_json) + self.changed_files = set([]) + if need_changed_files: + diff_url = github_event['pull_request']['diff_url'] + diff = urllib.request.urlopen(github_event['pull_request']['diff_url']) + diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) + self.changed_files = set([f.path for f in diff_object]) + def get_dict(self): return { 'sha': self.sha, diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 04fb166ccbd..233c7a45470 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -54,11 +54,10 @@ def process_result(result_folder): state, description = "error", "Failed to read test_results.tsv" return state, description, test_results, additional_files -def upload_results(s3_client, pr_number, commit_sha, state, description, test_results, additional_files): +def upload_results(s3_client, pr_number, commit_sha, test_results, additional_files): s3_path_prefix = f"{pr_number}/{commit_sha}/style_check" additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) - # Add link to help. Anchors in the docs must be adjusted accordingly. branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" branch_name = "master" if pr_number != 0: @@ -122,7 +121,7 @@ if __name__ == "__main__": subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) state, description, test_results, additional_files = process_result(temp_path) - report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, state, description, test_results, additional_files) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=state, target_url=report_url) From 0f2a1e957b2e398890e7d74d2d1d86bd8548ae34 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 19:34:14 +0300 Subject: [PATCH 094/950] Fix check --- .github/workflows/style-check.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index fe03f08127f..c41e531f2c2 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -31,7 +31,7 @@ jobs: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - DOCKER_ROBOT_PASSWORD: ${{ secrects.DOCKER_ROBOT_PASSWORD }} + DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} Style-Check: needs: DockerHubPush runs-on: [self-hosted] From c1d36e41f3e2fdb3a5d4bdd78e5eae6bc870e187 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 19:58:36 +0300 Subject: [PATCH 095/950] Docker images check --- tests/ci/docker_images_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 590935ab78b..8866df6e838 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -3,6 +3,7 @@ import subprocess import logging from report import create_test_html_report from s3_helper import S3Helper +import os NAME = "Push to Dockerhub (actions)" From 30d1f4c3adafcd495490f5d9fd7823fe6e552270 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:01:16 +0300 Subject: [PATCH 096/950] Docker images check --- tests/ci/docker_images_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 8866df6e838..7bf03427a78 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -3,6 +3,7 @@ import subprocess import logging from report import create_test_html_report from s3_helper import S3Helper +import json import os NAME = "Push to Dockerhub (actions)" From d2a76e32b8cb01fa836618a6a56ab84e50904e0b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:04:48 +0300 Subject: [PATCH 097/950] Docker image --- tests/ci/docker_images_check.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 7bf03427a78..df4e47705ed 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -5,6 +5,8 @@ from report import create_test_html_report from s3_helper import S3Helper import json import os +from pr_info import PRInfo +from github import Github NAME = "Push to Dockerhub (actions)" @@ -204,5 +206,6 @@ if __name__ == "__main__": url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) + gh = Github(os.getenv("GITHUB_TOKEN")) commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=status, target_url=url) From 5a750e05fd3865ddf88db7d9508f96779073a69a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:07:10 +0300 Subject: [PATCH 098/950] Fix --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index df4e47705ed..c49e88b1fc7 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -176,7 +176,7 @@ if __name__ == "__main__": event = json.load(event_file) pr_info = PRInfo(event) - changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, image_file_path) + changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") logging.info("Has changed images %s", ', '.join(changed_images)) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha versions = [str(pr_info.number), pr_commit_version] From 9c6723056295a6d2257e2d006545844f3468185f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:11:58 +0300 Subject: [PATCH 099/950] More fixes --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index c49e88b1fc7..91869f63d43 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -69,7 +69,7 @@ def get_changed_docker_images(pr_info, repo_path, image_file_path): no_dups_reversed.append(x) result = [(x, images_dict[x]['name']) for x in reversed(no_dups_reversed)] - logging.info("Changed docker images for PR %s @ %s: '%s'", pull_request.number, commit.sha, result) + logging.info("Changed docker images for PR %s @ %s: '%s'", pr_info.number, pr_info.sha, result) return result, dockerhub_repo_name def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_string): From ea6aa62f0e41165cdd2713e575db4d3e44a19fc8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:16:24 +0300 Subject: [PATCH 100/950] More fixes --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 91869f63d43..0e3eb54852e 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -201,7 +201,7 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) - s3_path_prefix = str(pull_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') + s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') status, test_results = process_test_results(s3_client, images_processing_result, s3_path_prefix) url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) From 0115b428ee85e366cbdcbf6cf76b45f3e453514e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:19:34 +0300 Subject: [PATCH 101/950] More fixes --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0e3eb54852e..854d56b8017 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -202,7 +202,7 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') - status, test_results = process_test_results(s3_client, images_processing_result, s3_path_prefix) + status, test_results = process_test_results(s3_helper, images_processing_result, s3_path_prefix) url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) From 0924da80d7e882f334cb77970ade644cea44727f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:22:27 +0300 Subject: [PATCH 102/950] More fixes --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 854d56b8017..0c2ebb52908 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -158,7 +158,7 @@ def upload_results(s3_client, pr_number, commit_sha, test_results): task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" - html_report = create_test_html_report(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url) + html_report = create_test_html_report(NAME, test_results, "https://hub.docker.com/u/clickhouse", task_url, branch_url, branch_name, commit_url) with open('report.html', 'w') as f: f.write(html_report) From 735716a2851118150ae2010520a7f3ce8d01aabc Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:25:12 +0300 Subject: [PATCH 103/950] More fixes --- tests/ci/docker_images_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0c2ebb52908..b046b8c9089 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -166,6 +166,11 @@ def upload_results(s3_client, pr_number, commit_sha, test_results): logging.info("Search result in url %s", url) return url +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) From 71dcf03a0f83c8126f5b9153ca202248d3893837 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:35:15 +0300 Subject: [PATCH 104/950] Create changed image --- docker/test/style/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/style/run.sh b/docker/test/style/run.sh index 424bfe71b15..0118e6df764 100755 --- a/docker/test/style/run.sh +++ b/docker/test/style/run.sh @@ -1,5 +1,7 @@ #!/bin/bash +# yaml check is not the best one + cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_output/check_status.tsv ./check-style -n |& tee /test_output/style_output.txt ./check-typos |& tee /test_output/typos_output.txt From 56499fb7ca56d6c8a51640db01883ae237b2789b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:47:04 +0300 Subject: [PATCH 105/950] Track changed files --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index b046b8c9089..6a9d1ba79c9 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -180,7 +180,7 @@ if __name__ == "__main__": with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: event = json.load(event_file) - pr_info = PRInfo(event) + pr_info = PRInfo(event, False, True) changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") logging.info("Has changed images %s", ', '.join(changed_images)) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha From 6245cc8b6aee5fe78ffa0282e13e67b626ab3404 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:48:39 +0300 Subject: [PATCH 106/950] Track changed files --- tests/ci/pr_info.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 0a8b0db2254..8feedb2d4d7 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -3,6 +3,7 @@ import requests import json import os import subprocess +import urllib from unidiff import PatchSet From 2ac210d63bc2abdd28b55123804737691c482a1a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:55:21 +0300 Subject: [PATCH 107/950] One more --- tests/ci/docker_images_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 6a9d1ba79c9..92353bb4386 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -103,7 +103,7 @@ def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_stri return True, build_log, push_log def process_single_image(versions, path_to_dockerfile_folder, image_name): - logging.info("Image will be pushed with versions %s", ', '.join(all_for_this_image)) + logging.info("Image will be pushed with versions %s", ', '.join(versions)) result = [] for ver in versions: for i in range(5): @@ -182,7 +182,7 @@ if __name__ == "__main__": pr_info = PRInfo(event, False, True) changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") - logging.info("Has changed images %s", ', '.join(changed_images)) + logging.info("Has changed images %s", ', '.join([str(image[0]) for image in changed_images])) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha versions = [str(pr_info.number), pr_commit_version] @@ -191,7 +191,7 @@ if __name__ == "__main__": images_processing_result = [] for rel_path, image_name in changed_images: full_path = os.path.join(repo_path, rel_path) - images_processing_result += process_single_image(versions, full_path, image) + images_processing_result += process_single_image(versions, full_path, image_name) if len(changed_images): description = "Updated " + ','.join([im[1] for im in images]) From d517ac3fce992f96a557bd0f59d7dd08b3a92f8a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 20:59:06 +0300 Subject: [PATCH 108/950] Fix build --- tests/ci/docker_images_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 92353bb4386..222b5ae3723 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -76,14 +76,14 @@ def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_stri logging.info("Building docker image %s with version %s from path %s", image_name, version_string, path_to_dockerfile_folder) build_log = None push_log = None - with open('build_log_' + str(image_name) + "_" + version_string, 'w') as pl: + with open('build_log_' + str(image_name).replace('/', '_') + "_" + version_string, 'w') as pl: cmd = "docker build --network=host -t {im}:{ver} {path}".format(im=image_name, ver=version_string, path=path_to_dockerfile_folder) retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() build_log = str(c.name) if retcode != 0: return False, build_log, None - with open('tag_log_' + str(image_name) + "_" + version_string, 'w') as pl: + with open('tag_log_' + str(image_name).replace('/', '_') + "_" + version_string, 'w') as pl: cmd = "docker build --network=host -t {im} {path}".format(im=image_name, path=path_to_dockerfile_folder) retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() build_log = str(pl.name) @@ -92,7 +92,7 @@ def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_stri logging.info("Pushing image %s to dockerhub", image_name) - with open('push_log_' + str(image_name) + "_" + version_string, 'w') as pl: + with open('push_log_' + str(image_name).replace('/', '_') + "_" + version_string, 'w') as pl: cmd = "docker push {im}:{ver}".format(im=image_name, ver=version_string) retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() push_log = str(pl.stdout.path) From 3fedd11c0bf41480be8fe1aef0c9b19261916381 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:01:36 +0300 Subject: [PATCH 109/950] Fix accident changes: --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 222b5ae3723..06162fe4624 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -79,7 +79,7 @@ def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_stri with open('build_log_' + str(image_name).replace('/', '_') + "_" + version_string, 'w') as pl: cmd = "docker build --network=host -t {im}:{ver} {path}".format(im=image_name, ver=version_string, path=path_to_dockerfile_folder) retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() - build_log = str(c.name) + build_log = str(pl.name) if retcode != 0: return False, build_log, None From 7a4ff98612d8854de9da90cd80121b5d57bae43e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:05:02 +0300 Subject: [PATCH 110/950] Fix accident changes: --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 06162fe4624..c4532e449f5 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -95,7 +95,7 @@ def build_and_push_one_image(path_to_dockerfile_folder, image_name, version_stri with open('push_log_' + str(image_name).replace('/', '_') + "_" + version_string, 'w') as pl: cmd = "docker push {im}:{ver}".format(im=image_name, ver=version_string) retcode = subprocess.Popen(cmd, shell=True, stderr=pl, stdout=pl).wait() - push_log = str(pl.stdout.path) + push_log = str(pl.name) if retcode != 0: return False, build_log, push_log From 340d24d07b41e108ebbee1ff7104221578211758 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:07:42 +0300 Subject: [PATCH 111/950] Fix accident changes: --- tests/ci/docker_images_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index c4532e449f5..0e8414e6df5 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -194,7 +194,7 @@ if __name__ == "__main__": images_processing_result += process_single_image(versions, full_path, image_name) if len(changed_images): - description = "Updated " + ','.join([im[1] for im in images]) + description = "Updated " + ','.join([im[1] for im in changed_images]) else: description = "Nothing to update" From b5aca2265be2a996f30ea5a26f0beea55d49b0a3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:26:48 +0300 Subject: [PATCH 112/950] Trying to path images --- .github/workflows/style-check.yml | 10 ++++++++++ tests/ci/docker_images_check.py | 7 +++++++ tests/ci/style_check.py | 15 +++++++++++++-- 3 files changed, 30 insertions(+), 2 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index c41e531f2c2..4bfffcf9f15 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -32,10 +32,20 @@ jobs: YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/changed_images.json Style-Check: needs: DockerHubPush runs-on: [self-hosted] steps: + - name: Download math result for job 1 + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/changed_images.json - name: Check out repository code uses: actions/checkout@v2 - name: Style Check diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0e8414e6df5..284406466a9 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -184,20 +184,24 @@ if __name__ == "__main__": changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") logging.info("Has changed images %s", ', '.join([str(image[0]) for image in changed_images])) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha + versions = [str(pr_info.number), pr_commit_version] subprocess.check_output("docker login --username 'robotclickhouse' --password '{}'".format(dockerhub_password), shell=True) + result_images = {} images_processing_result = [] for rel_path, image_name in changed_images: full_path = os.path.join(repo_path, rel_path) images_processing_result += process_single_image(versions, full_path, image_name) + result_images[image_name] = pr_commit_version if len(changed_images): description = "Updated " + ','.join([im[1] for im in changed_images]) else: description = "Nothing to update" + if len(description) >= 140: description = description[:136] + "..." @@ -214,3 +218,6 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=status, target_url=url) + + with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: + json.dump(result_images, images_file) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 233c7a45470..9721fe60b18 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -107,7 +107,18 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) - docker_image_version = os.getenv("DOCKER_IMAGE_VERSION", "latest") + images_path = os.path.join(temp_path, 'changed_images.json') + docker_image = 'clickhouse/style-check' + if os.path.exists(images_path): + logging.info("Images file exists") + with open(images_path, 'r') as images_fd: + images = json.loads(images_fd) + logging.info("Got images %s", images) + if 'clickhouse/style-check' in images: + docker_image += ':' + images['clickhouse/style-check'] + + logging.info("Got docker image %s", docker_image) + if not aws_secret_key_id or not aws_secret_key: logging.info("No secrets, will not upload anything to S3") @@ -119,7 +130,7 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) - subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output clickhouse/style-test:{docker_image_version}", shell=True) + subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) From fc43998c944c42bfc4c9c6ab1346d121f705c5cf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:48:06 +0300 Subject: [PATCH 113/950] Fixes --- .github/workflows/style-check.yml | 4 ++-- tests/ci/docker_images_check.py | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 4bfffcf9f15..da2b433a62b 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -36,7 +36,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/changed_images.json + path: ${{ runner.temp }}/docker_images_check/changed_images.json Style-Check: needs: DockerHubPush runs-on: [self-hosted] @@ -45,7 +45,7 @@ jobs: uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/changed_images.json + path: ${{ runner.temp }}/style_check/changed_images.json - name: Check out repository code uses: actions/checkout@v2 - name: Style Check diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 284406466a9..254efa9e94a 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -177,6 +177,12 @@ if __name__ == "__main__": temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') dockerhub_password = os.getenv('DOCKER_ROBOT_PASSWORD') + if os.path.exists(temp_path): + shutil.rmtree(temp_path) + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: event = json.load(event_file) From e9364fc0752309c212782eadedef24eed6129ece Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:52:37 +0300 Subject: [PATCH 114/950] One more time --- .github/workflows/style-check.yml | 2 +- tests/ci/docker_images_check.py | 3 --- tests/ci/style_check.py | 9 +++------ 3 files changed, 4 insertions(+), 10 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index da2b433a62b..1a7ceb323cb 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -45,7 +45,7 @@ jobs: uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/style_check/changed_images.json + path: ${{ runner.temp }}/style_check - name: Check out repository code uses: actions/checkout@v2 - name: Style Check diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 254efa9e94a..bb64474c878 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -177,9 +177,6 @@ if __name__ == "__main__": temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') dockerhub_password = os.getenv('DOCKER_ROBOT_PASSWORD') - if os.path.exists(temp_path): - shutil.rmtree(temp_path) - if not os.path.exists(temp_path): os.makedirs(temp_path) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9721fe60b18..008e3e88490 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -102,6 +102,9 @@ if __name__ == "__main__": event = json.load(event_file) pr_info = PRInfo(event) + if not os.path.exists(temp_path): + os.makedirs(temp_path) + aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") @@ -124,12 +127,6 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) - if os.path.exists(temp_path): - shutil.rmtree(temp_path) - - if not os.path.exists(temp_path): - os.makedirs(temp_path) - subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) From fbd7cc4f690fdc693ee83789e2cf68fa6318e67d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:54:20 +0300 Subject: [PATCH 115/950] Followup --- .github/workflows/style-check.yml | 2 +- tests/ci/style_check.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 1a7ceb323cb..cde033d4d91 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -41,7 +41,7 @@ jobs: needs: DockerHubPush runs-on: [self-hosted] steps: - - name: Download math result for job 1 + - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 008e3e88490..9ab27bb22a5 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -115,7 +115,7 @@ if __name__ == "__main__": if os.path.exists(images_path): logging.info("Images file exists") with open(images_path, 'r') as images_fd: - images = json.loads(images_fd) + images = json.load(images_fd) logging.info("Got images %s", images) if 'clickhouse/style-check' in images: docker_image += ':' + images['clickhouse/style-check'] From 25171f8e0768941d4c96e1a850042a63cdbb5ea9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Sep 2021 21:57:48 +0300 Subject: [PATCH 116/950] Almost there --- tests/ci/style_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9ab27bb22a5..9ff9d7e54ac 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -111,14 +111,14 @@ if __name__ == "__main__": gh = Github(os.getenv("GITHUB_TOKEN")) images_path = os.path.join(temp_path, 'changed_images.json') - docker_image = 'clickhouse/style-check' + docker_image = 'clickhouse/style-test' if os.path.exists(images_path): logging.info("Images file exists") with open(images_path, 'r') as images_fd: images = json.load(images_fd) logging.info("Got images %s", images) - if 'clickhouse/style-check' in images: - docker_image += ':' + images['clickhouse/style-check'] + if 'clickhouse/style-test' in images: + docker_image += ':' + images['clickhouse/style-test'] logging.info("Got docker image %s", docker_image) From bc1a0b79354626b9e8d6d4d5a5b13a47d9c425ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 11:33:55 +0300 Subject: [PATCH 117/950] Branding? --- .github/workflows/style-check.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index cde033d4d91..08c96d2398d 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -1,4 +1,7 @@ name: Ligthweight GithubActions +branding: + icon: 'award' + color: 'green' on: # yamllint disable-line rule:truthy pull_request: types: From e471cdce4999e6b618eb617dbb969142cc8ea265 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 12:26:49 +0300 Subject: [PATCH 118/950] Trying reports --- .github/workflows/style-check.yml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 08c96d2398d..5f8032b43d3 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -1,7 +1,4 @@ name: Ligthweight GithubActions -branding: - icon: 'award' - color: 'green' on: # yamllint disable-line rule:truthy pull_request: types: @@ -22,6 +19,8 @@ jobs: run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + - name: Just Checking + run: echo "::notice ::Hello world" DockerHubPush: needs: CheckLabels runs-on: [self-hosted] From cbc34c66b0577b6a55c4e22413426331cb99fb33 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 12:33:38 +0300 Subject: [PATCH 119/950] Trying annotations --- .github/workflows/style-check.yml | 2 -- tests/ci/docker_images_check.py | 2 ++ tests/ci/style_check.py | 4 +--- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 5f8032b43d3..cde033d4d91 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -19,8 +19,6 @@ jobs: run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - - name: Just Checking - run: echo "::notice ::Hello world" DockerHubPush: needs: CheckLabels runs-on: [self-hosted] diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index bb64474c878..a7901b5fda8 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -224,3 +224,5 @@ if __name__ == "__main__": with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: json.dump(result_images, images_file) + + print("::notice ::Report url: {}".format(url)) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9ff9d7e54ac..4a75eee70ee 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -130,6 +130,4 @@ if __name__ == "__main__": subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) - - commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description=description, state=state, target_url=report_url) + print("::notice ::Report url: {}".format(report_url)) From a2772f4304757ef6da7a4b466820da9065a5dd9f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 12:37:39 +0300 Subject: [PATCH 120/950] Maybe supports html --- tests/ci/docker_images_check.py | 2 +- tests/ci/run_check.py | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index a7901b5fda8..639f19e6973 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -225,4 +225,4 @@ if __name__ == "__main__": with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: json.dump(result_images, images_file) - print("::notice ::Report url: {}".format(url)) + print("::notice ::

Report url

: {}".format(url)) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 26e648dae90..788bfc5b5b1 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -114,7 +114,9 @@ if __name__ == "__main__": commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: + print("::notice ::

Cannot run

") commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) else: + print("::notice ::

Can run

") commit.create_status(context=NAME, description=description, state="pending", target_url=url) From dc3396a2cfc3bf6f676f32cf6feb15197200e936 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 12:41:13 +0300 Subject: [PATCH 121/950] Branding --- .github/workflows/style-check.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index cde033d4d91..3f691242acc 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -11,6 +11,9 @@ on: # yamllint disable-line rule:truthy - master jobs: CheckLabels: + branding: + icon: 'award' + color: 'green' runs-on: [self-hosted] steps: - name: Check out repository code @@ -19,6 +22,7 @@ jobs: run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + run: echo "::notice ::Hello world" DockerHubPush: needs: CheckLabels runs-on: [self-hosted] From e439532f905ad14c8f7e17b65e328059328a127d Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 12:43:19 +0300 Subject: [PATCH 122/950] Followup --- .github/workflows/style-check.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 3f691242acc..6d8013c2e8a 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -22,7 +22,6 @@ jobs: run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - run: echo "::notice ::Hello world" DockerHubPush: needs: CheckLabels runs-on: [self-hosted] From 42da0d71788defe521585ec4086a50beef3a0a4b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:04:37 +0300 Subject: [PATCH 123/950] Trying split actions --- .github/workflows/docker-hub-action.yml | 18 ++++++++++++++++++ .github/workflows/style-check.yml | 7 +------ 2 files changed, 19 insertions(+), 6 deletions(-) create mode 100644 .github/workflows/docker-hub-action.yml diff --git a/.github/workflows/docker-hub-action.yml b/.github/workflows/docker-hub-action.yml new file mode 100644 index 00000000000..924c4692e57 --- /dev/null +++ b/.github/workflows/docker-hub-action.yml @@ -0,0 +1,18 @@ +name: 'DockerHubPush' +description: 'Action push images to dockerhub' +runs: + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py + env: + YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} + YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docker_images_check/changed_images.json diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 6d8013c2e8a..72ea5a8a27d 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -11,9 +11,6 @@ on: # yamllint disable-line rule:truthy - master jobs: CheckLabels: - branding: - icon: 'award' - color: 'green' runs-on: [self-hosted] steps: - name: Check out repository code @@ -26,10 +23,8 @@ jobs: needs: CheckLabels runs-on: [self-hosted] steps: - - name: Check out repository code - uses: actions/checkout@v2 - name: Images check - run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py + uses: .github/workflows env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} From fd0a4bdd67f70dff65f11b9408b9e7c19632565a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:05:59 +0300 Subject: [PATCH 124/950] Add on --- .github/workflows/docker-hub-action.yml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/.github/workflows/docker-hub-action.yml b/.github/workflows/docker-hub-action.yml index 924c4692e57..66d969ec29f 100644 --- a/.github/workflows/docker-hub-action.yml +++ b/.github/workflows/docker-hub-action.yml @@ -1,5 +1,15 @@ name: 'DockerHubPush' description: 'Action push images to dockerhub' +on: # yamllint disable-line rule:truthy + pull_request: + types: + - labeled + - unlabeled + - synchronize + - reopened + - opened + branches: + - master runs: steps: - name: Check out repository code From bafad0fb09962806ca7596655c52b3cc0f7e7576 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:07:10 +0300 Subject: [PATCH 125/950] Trying --- .github/{workflows => actions}/docker-hub-action.yml | 0 .github/workflows/style-check.yml | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename .github/{workflows => actions}/docker-hub-action.yml (100%) diff --git a/.github/workflows/docker-hub-action.yml b/.github/actions/docker-hub-action.yml similarity index 100% rename from .github/workflows/docker-hub-action.yml rename to .github/actions/docker-hub-action.yml diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 72ea5a8a27d..72d6d104a2b 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -24,7 +24,7 @@ jobs: runs-on: [self-hosted] steps: - name: Images check - uses: .github/workflows + uses: .github/actions env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} From 5d3c92dcaf0cc5d0263e131f7f05ada78e97d17a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:11:35 +0300 Subject: [PATCH 126/950] No idea --- .github/workflows/style-check.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 72d6d104a2b..29943ce30ee 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -23,6 +23,8 @@ jobs: needs: CheckLabels runs-on: [self-hosted] steps: + - name: Check out repository code + uses: actions/checkout@v2 - name: Images check uses: .github/actions env: From 0c68a7c1a9fc13441dd4040a225b2e9de5333cf7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:15:14 +0300 Subject: [PATCH 127/950] Don't give up --- .../{docker-hub-action.yml => docker-hub-push/action.yml} | 0 .github/workflows/style-check.yml | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename .github/actions/{docker-hub-action.yml => docker-hub-push/action.yml} (100%) diff --git a/.github/actions/docker-hub-action.yml b/.github/actions/docker-hub-push/action.yml similarity index 100% rename from .github/actions/docker-hub-action.yml rename to .github/actions/docker-hub-push/action.yml diff --git a/.github/workflows/style-check.yml b/.github/workflows/style-check.yml index 29943ce30ee..37f48fdfc29 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/style-check.yml @@ -26,7 +26,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Images check - uses: .github/actions + uses: ./.github/actions/docker-hub-push env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} From 52d6d44370a10498730f90dbe86f72977948646b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:20:39 +0300 Subject: [PATCH 128/950] Shell bash --- .github/actions/docker-hub-push/action.yml | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/.github/actions/docker-hub-push/action.yml b/.github/actions/docker-hub-push/action.yml index 66d969ec29f..9352d158a3b 100644 --- a/.github/actions/docker-hub-push/action.yml +++ b/.github/actions/docker-hub-push/action.yml @@ -16,11 +16,7 @@ runs: uses: actions/checkout@v2 - name: Images check run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py - env: - YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} - YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} + shell: bash - name: Upload images files to artifacts uses: actions/upload-artifact@v2 with: From 1bfcbe281cceeba679454fff37e1d3a1f09ff107 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:26:45 +0300 Subject: [PATCH 129/950] Trying run --- .github/actions/docker-hub-push/action.yml | 24 ------------------- .../workflows/{style-check.yml => main.yml} | 2 +- tests/ci/run_check.py | 4 ++-- tests/ci/style_check.py | 2 +- 4 files changed, 4 insertions(+), 28 deletions(-) delete mode 100644 .github/actions/docker-hub-push/action.yml rename .github/workflows/{style-check.yml => main.yml} (95%) diff --git a/.github/actions/docker-hub-push/action.yml b/.github/actions/docker-hub-push/action.yml deleted file mode 100644 index 9352d158a3b..00000000000 --- a/.github/actions/docker-hub-push/action.yml +++ /dev/null @@ -1,24 +0,0 @@ -name: 'DockerHubPush' -description: 'Action push images to dockerhub' -on: # yamllint disable-line rule:truthy - pull_request: - types: - - labeled - - unlabeled - - synchronize - - reopened - - opened - branches: - - master -runs: - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Images check - run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py - shell: bash - - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/docker_images_check/changed_images.json diff --git a/.github/workflows/style-check.yml b/.github/workflows/main.yml similarity index 95% rename from .github/workflows/style-check.yml rename to .github/workflows/main.yml index 37f48fdfc29..cde033d4d91 100644 --- a/.github/workflows/style-check.yml +++ b/.github/workflows/main.yml @@ -26,7 +26,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Images check - uses: ./.github/actions/docker-hub-push + run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 788bfc5b5b1..87dc21beda2 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -114,9 +114,9 @@ if __name__ == "__main__": commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: - print("::notice ::

Cannot run

") + print("::notice ::**Cannot run**") commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) else: - print("::notice ::

Can run

") + print("::notice ::**Can run**") commit.create_status(context=NAME, description=description, state="pending", target_url=url) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 4a75eee70ee..3438e40a5b4 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -130,4 +130,4 @@ if __name__ == "__main__": subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) - print("::notice ::Report url: {}".format(report_url)) + print("::notice ::Report *url*: {} and one more [test]({})".format(report_url, report_url)) From 8b0a85fd90dfa36a3619e34a972e7481f3aed704 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:39:36 +0300 Subject: [PATCH 130/950] Remove --- tests/ci/docker_images_check.py | 2 +- tests/ci/run_check.py | 4 ++-- tests/ci/style_check.py | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 639f19e6973..a7901b5fda8 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -225,4 +225,4 @@ if __name__ == "__main__": with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: json.dump(result_images, images_file) - print("::notice ::

Report url

: {}".format(url)) + print("::notice ::Report url: {}".format(url)) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 87dc21beda2..70b3ae2ac07 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -114,9 +114,9 @@ if __name__ == "__main__": commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: - print("::notice ::**Cannot run**") + print("::notice ::Cannot run") commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) else: - print("::notice ::**Can run**") + print("::notice ::Can run") commit.create_status(context=NAME, description=description, state="pending", target_url=url) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 3438e40a5b4..4a75eee70ee 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -130,4 +130,4 @@ if __name__ == "__main__": subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) - print("::notice ::Report *url*: {} and one more [test]({})".format(report_url, report_url)) + print("::notice ::Report url: {}".format(report_url)) From 2a74979c3aecf325d5233b97fde7658476fdc55d Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:52:04 +0300 Subject: [PATCH 131/950] Trying output --- tests/ci/docker_images_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index a7901b5fda8..b9353a0a44f 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -177,6 +177,9 @@ if __name__ == "__main__": temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') dockerhub_password = os.getenv('DOCKER_ROBOT_PASSWORD') + if os.path.exists(temp_path): + shutil.rmtree(temp_path) + if not os.path.exists(temp_path): os.makedirs(temp_path) @@ -226,3 +229,5 @@ if __name__ == "__main__": json.dump(result_images, images_file) print("::notice ::Report url: {}".format(url)) + print("::set-output name=url_output::\"{}\"".format(url)) +) From 7aac1e29b9931b07185dc84c2d76179a135c5aa6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 13:54:57 +0300 Subject: [PATCH 132/950] Remove --- tests/ci/docker_images_check.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index b9353a0a44f..bff229e15ff 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -230,4 +230,3 @@ if __name__ == "__main__": print("::notice ::Report url: {}".format(url)) print("::set-output name=url_output::\"{}\"".format(url)) -) From 755e4d2e9e6d8c973dd57eb4faabe914a929adcc Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 14:17:26 +0300 Subject: [PATCH 133/950] Actions --- tests/ci/style_check.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 4a75eee70ee..47b12c11173 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -131,3 +131,5 @@ if __name__ == "__main__": state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) print("::notice ::Report url: {}".format(report_url)) + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=status, target_url=url) From ec7d83c9e080353b20cb68800a483596af222ee2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 14:41:07 +0300 Subject: [PATCH 134/950] Fix style check --- tests/ci/style_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 47b12c11173..5b2c2258585 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -132,4 +132,4 @@ if __name__ == "__main__": report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) print("::notice ::Report url: {}".format(report_url)) commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description=description, state=status, target_url=url) + commit.create_status(context=NAME, description=description, state=state, target_url=report_url) From d25b8881e4c30163abc1099db89365740e1ff2f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 15:12:58 +0300 Subject: [PATCH 135/950] More copypaste --- .github/workflows/main.yml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index cde033d4d91..2ff22f0fee6 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -54,3 +54,21 @@ jobs: YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py + PVS-Check: + needs: DockerHubPush + runs-on: [self-hosted] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/pvs_check + - name: Check out repository code + uses: actions/checkout@v2 + - name: PVS Check + env: + YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} + YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + PVS_STUDIO_KEY: ${{ secrets.PVS_STUDIO_KEY }} + run: cd $GITHUB_WORKSPACE/tests/ci && python3 pvs_check.py From 1029cb3095634d4a883ffbc870f6809390e4631c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 15:17:47 +0300 Subject: [PATCH 136/950] Add PVS check --- tests/ci/pvs_check.py | 137 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 137 insertions(+) create mode 100644 tests/ci/pvs_check.py diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py new file mode 100644 index 00000000000..4c2957770e9 --- /dev/null +++ b/tests/ci/pvs_check.py @@ -0,0 +1,137 @@ +#!/usr/bin/env python3 +import subprocess +import os +import json +import logging +from github import Github +from report import create_test_html_report +from s3_helper import S3Helper +from pr_info import PRInfo +import shutil +import sys + +NAME = 'PVS Studio (actions)' +LICENSE_NAME = 'Free license: ClickHouse, Yandex' +HTML_REPORT_FOLDER = 'pvs-studio-html-report' +TXT_REPORT_NAME = 'pvs-studio-task-report.txt' + +def process_logs(s3_client, additional_logs, s3_path_prefix): + additional_urls = [] + for log_path in additional_logs: + if log_path: + additional_urls.append( + s3_client.upload_test_report_to_s3( + log_path, + s3_path_prefix + "/" + os.path.basename(log_path))) + + return additional_urls + +def _process_txt_report(self, path): + warnings = [] + errors = [] + with open(path, 'r') as report_file: + for line in report_file: + if 'viva64' in line: + continue + elif 'warn' in line: + warnings.append(':'.join(line.split('\t')[0:2])) + elif 'err' in line: + errors.append(':'.join(line.split('\t')[0:2])) + return warnings, errors + +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + +def upload_results(s3_client, pr_number, commit_sha, test_results, additional_files): + s3_path_prefix = str(pr_number) + "/" + commit_sha + "/" + NAME.lower().replace(' ', '_') + additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) + + branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_name = "master" + if pr_number != 0: + branch_name = "PR #{}".format(pr_number) + branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(pr_number) + commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" + + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + + raw_log_url = additional_urls[0] + additional_urls.pop(0) + + html_report = create_test_html_report(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + with open('report.html', 'w') as f: + f.write(html_report) + + url = s3_client.upload_test_report_to_s3('report.html', s3_path_prefix + ".html") + logging.info("Search result in url %s", url) + return url + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) + temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'pvs_check') + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + pr_info = PRInfo(event) + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") + aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") + + gh = Github(os.getenv("GITHUB_TOKEN")) + + images_path = os.path.join(temp_path, 'changed_images.json') + docker_image = 'clickhouse/pvs-test' + if os.path.exists(images_path): + logging.info("Images file exists") + with open(images_path, 'r') as images_fd: + images = json.load(images_fd) + logging.info("Got images %s", images) + if 'clickhouse/pvs-test' in images: + docker_image += ':' + images['clickhouse/pvs-test'] + + logging.info("Got docker image %s", docker_image) + + if not aws_secret_key_id or not aws_secret_key: + logging.info("No secrets, will not upload anything to S3") + + s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + + licence_key = os.getenv('PVS_STUDIO_KEY') + cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENSE_NAME='{LICENSE_NAME}' -e LICENCE_KEY='{licence_key}' -e CC=clang-11 -e CXX=clang++-11 {docker_image}" + + subprocess.check_output(cmd, shell=True) + + s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') + html_urls = self.s3_client.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) + index_html = None + + commit = get_commit(gh, pr_info.sha) + for url in html_urls: + if 'index.html' in url: + index_html = 'HTML report'.format(url) + break + + if not index_html: + commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + sys.exit(1) + + txt_report = os.path.join(temp_path, TXT_REPORT_NAME) + warnings, errors = _process_txt_report(txt_report) + errors = errors + warnings + + status = 'success' + test_results = [(index_html, "Look at the report")] + description = "Total errors {}".format(len(errors)) + additional_logs = [txt_report, os.path.join(temp_path, 'pvs-studio.log')] + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs) + + print("::notice ::Report url: {}".format(report_url)) + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=status, target_url=report_url) From 0ea203da310a753f456fb75329d22baceacf7b05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 15:24:24 +0300 Subject: [PATCH 137/950] Checkout submodules for PVS --- .github/workflows/main.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 2ff22f0fee6..025ceea8c50 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -65,6 +65,8 @@ jobs: path: ${{ runner.temp }}/pvs_check - name: Check out repository code uses: actions/checkout@v2 + with: + submodules: true - name: PVS Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} From a895375893f7fd1575d4ec7cd6a430211f6e81f0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 15:28:11 +0300 Subject: [PATCH 138/950] Something strange --- tests/ci/pvs_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 4c2957770e9..9aabb5f734c 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -127,7 +127,7 @@ if __name__ == "__main__": errors = errors + warnings status = 'success' - test_results = [(index_html, "Look at the report")] + test_results = [(index_html, "Look at the report"), ("Errors count not checked", "OK")] description = "Total errors {}".format(len(errors)) additional_logs = [txt_report, os.path.join(temp_path, 'pvs-studio.log')] report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs) From a70d4d13dfec21907a58dea4706013878c5c017c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 15:44:42 +0300 Subject: [PATCH 139/950] Trying one more time --- .github/workflows/main.yml | 2 +- tests/ci/pvs_check.py | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 025ceea8c50..4e5c2ed19e9 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -66,7 +66,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: true + submodules: 'recursive' - name: PVS Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 9aabb5f734c..1becffedac2 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -104,9 +104,13 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) licence_key = os.getenv('PVS_STUDIO_KEY') - cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENSE_NAME='{LICENSE_NAME}' -e LICENCE_KEY='{licence_key}' -e CC=clang-11 -e CXX=clang++-11 {docker_image}" + cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENSE_NAME='{LICENSE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" - subprocess.check_output(cmd, shell=True) + try: + subprocess.check_output(cmd, shell=True) + except: + commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + sys.exit(1) s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') html_urls = self.s3_client.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) From fd4afa236b3756888b4018942a92b6cc2c28f925 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 16:02:40 +0300 Subject: [PATCH 140/950] Fix licence name --- tests/ci/pvs_check.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 1becffedac2..b52b0b32f1b 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -104,7 +104,8 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) licence_key = os.getenv('PVS_STUDIO_KEY') - cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENSE_NAME='{LICENSE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" + cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENSE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" + commit = get_commit(gh, pr_info.sha) try: subprocess.check_output(cmd, shell=True) @@ -116,7 +117,6 @@ if __name__ == "__main__": html_urls = self.s3_client.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) index_html = None - commit = get_commit(gh, pr_info.sha) for url in html_urls: if 'index.html' in url: index_html = 'HTML report'.format(url) From ed07b085de10b3f5131aec0e80ffc7b51df6382f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 16:04:47 +0300 Subject: [PATCH 141/950] Trying to fix --- .github/workflows/main.yml | 1 + tests/ci/pvs_check.py | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 4e5c2ed19e9..245f76eb3b6 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -67,6 +67,7 @@ jobs: uses: actions/checkout@v2 with: submodules: 'recursive' + path: 'repo_with_submodules' - name: PVS Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index b52b0b32f1b..516f4c16e41 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -71,7 +71,7 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) + repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")), "repo_with_submodules") temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'pvs_check') with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: From e3e69825183322d685c90d9909fdd151c81a1764 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 16:05:04 +0300 Subject: [PATCH 142/950] Followup --- tests/ci/pvs_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 516f4c16e41..3778a6e3110 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -104,7 +104,7 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) licence_key = os.getenv('PVS_STUDIO_KEY') - cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENSE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" + cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENCE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" commit = get_commit(gh, pr_info.sha) try: From 68480a659e942e276d248e073f23d164c3f1ffdf Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 16:14:24 +0300 Subject: [PATCH 143/950] Followup --- tests/ci/pvs_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 3778a6e3110..c7f07a34e32 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -11,7 +11,7 @@ import shutil import sys NAME = 'PVS Studio (actions)' -LICENSE_NAME = 'Free license: ClickHouse, Yandex' +LICENCE_NAME = 'Free license: ClickHouse, Yandex' HTML_REPORT_FOLDER = 'pvs-studio-html-report' TXT_REPORT_NAME = 'pvs-studio-task-report.txt' From 736673bf08ca899c7e42280ebe598e58a516b57f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 16:52:16 +0300 Subject: [PATCH 144/950] Moar --- tests/ci/pvs_check.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index c7f07a34e32..75febd9cd49 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -26,7 +26,7 @@ def process_logs(s3_client, additional_logs, s3_path_prefix): return additional_urls -def _process_txt_report(self, path): +def _process_txt_report(path): warnings = [] errors = [] with open(path, 'r') as report_file: @@ -114,7 +114,7 @@ if __name__ == "__main__": sys.exit(1) s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') - html_urls = self.s3_client.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) + html_urls = s3_helper.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) index_html = None for url in html_urls: From 8b1be85bcbe771e7c28fd3bd4a4b199ff202705a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 18:51:43 +0300 Subject: [PATCH 145/950] Trying other way --- .github/workflows/main.yml | 2 ++ tests/ci/pvs_check.py | 46 +++++++++++++++++++++----------------- tests/ci/s3_helper.py | 2 +- tests/ci/style_check.py | 2 +- 4 files changed, 29 insertions(+), 23 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 245f76eb3b6..a1d6cf05fd6 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -48,6 +48,8 @@ jobs: path: ${{ runner.temp }}/style_check - name: Check out repository code uses: actions/checkout@v2 + with: + path: 'repo_without_submodules' - name: Style Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 75febd9cd49..8bc6df632f2 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -113,29 +113,33 @@ if __name__ == "__main__": commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") sys.exit(1) - s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') - html_urls = s3_helper.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) - index_html = None + try: + s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') + html_urls = s3_helper.upload_test_folder_to_s3(os.path.join(temp_path, HTML_REPORT_FOLDER), s3_path_prefix) + index_html = None - for url in html_urls: - if 'index.html' in url: - index_html = 'HTML report'.format(url) - break + for url in html_urls: + if 'index.html' in url: + index_html = 'HTML report'.format(url) + break - if not index_html: - commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") - sys.exit(1) + if not index_html: + commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + sys.exit(1) - txt_report = os.path.join(temp_path, TXT_REPORT_NAME) - warnings, errors = _process_txt_report(txt_report) - errors = errors + warnings + txt_report = os.path.join(temp_path, TXT_REPORT_NAME) + warnings, errors = _process_txt_report(txt_report) + errors = errors + warnings - status = 'success' - test_results = [(index_html, "Look at the report"), ("Errors count not checked", "OK")] - description = "Total errors {}".format(len(errors)) - additional_logs = [txt_report, os.path.join(temp_path, 'pvs-studio.log')] - report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs) + status = 'success' + test_results = [(index_html, "Look at the report"), ("Errors count not checked", "OK")] + description = "Total errors {}".format(len(errors)) + additional_logs = [txt_report, os.path.join(temp_path, 'pvs-studio.log')] + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs) - print("::notice ::Report url: {}".format(report_url)) - commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description=description, state=status, target_url=report_url) + print("::notice ::Report url: {}".format(report_url)) + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=status, target_url=report_url) + except Exception as ex: + print("Got an exception", ex) + sys.exit(1) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 8a170da44f8..b9ae0de6e02 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -71,7 +71,7 @@ class S3Helper(object): if not files: return [] - p = Pool(min(len(files), 30)) + p = Pool(min(len(files), 5)) def task(file_name): full_fs_path = os.path.join(folder_path, file_name) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 5b2c2258585..f41120f7de7 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -95,7 +95,7 @@ def update_check_with_curl(check_id): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) + repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")), 'repo_without_submodules') temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'style_check') with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: From c6f9c8e7ba992f4eab909f5b94d179442550ec06 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 18:54:40 +0300 Subject: [PATCH 146/950] Missed file --- tests/ci/docker_images_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index bff229e15ff..141d075cc6d 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -7,6 +7,7 @@ import json import os from pr_info import PRInfo from github import Github +import shutil NAME = "Push to Dockerhub (actions)" From fce1d7e156502cf71a16b0f8ff6001c209bfd44c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Sep 2021 22:35:38 +0300 Subject: [PATCH 147/950] Fix stupid bug --- tests/ci/pvs_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 8bc6df632f2..18cee175970 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -123,9 +123,9 @@ if __name__ == "__main__": index_html = 'HTML report'.format(url) break - if not index_html: - commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") - sys.exit(1) + if not index_html: + commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + sys.exit(1) txt_report = os.path.join(temp_path, TXT_REPORT_NAME) warnings, errors = _process_txt_report(txt_report) From 6556e77eb42771791e5eb33840433a4772faf4ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 10:51:24 +0300 Subject: [PATCH 148/950] Test From d0d4318624e7ddc4f9c24ac4ea56c44c875b94d4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 11:20:56 +0300 Subject: [PATCH 149/950] Use correct user --- .github/workflows/main.yml | 3 --- tests/ci/pvs_check.py | 11 +++++++++-- tests/ci/style_check.py | 4 ++-- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index a1d6cf05fd6..4e5c2ed19e9 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -48,8 +48,6 @@ jobs: path: ${{ runner.temp }}/style_check - name: Check out repository code uses: actions/checkout@v2 - with: - path: 'repo_without_submodules' - name: Style Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} @@ -69,7 +67,6 @@ jobs: uses: actions/checkout@v2 with: submodules: 'recursive' - path: 'repo_with_submodules' - name: PVS Check env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 18cee175970..f68e5ca8210 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -71,7 +71,7 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")), "repo_with_submodules") + repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../"))) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'pvs_check') with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: @@ -81,6 +81,13 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) + new_repo_path = os.path.join(temp_path, repo_path) + if os.path.exists(new_repo_path): + shutil.rmtree(new_repo_path) + shutil.copytree(repo_path, temp_path) + # this check modify repository so copy it to the temp directory + repo_path = new_repo_path + aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") @@ -104,7 +111,7 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) licence_key = os.getenv('PVS_STUDIO_KEY') - cmd = f"docker run --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENCE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" + cmd = f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENCE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" commit = get_commit(gh, pr_info.sha) try: diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index f41120f7de7..2af8514fbfc 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -95,7 +95,7 @@ def update_check_with_curl(check_id): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")), 'repo_without_submodules') + repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../"))) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'style_check') with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: @@ -127,7 +127,7 @@ if __name__ == "__main__": s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) - subprocess.check_output(f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) + subprocess.check_output(f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) print("::notice ::Report url: {}".format(report_url)) From 18f282858e9bda66ba314fc3f6a9636fdb197b47 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 12:41:31 +0300 Subject: [PATCH 150/950] Add logging --- tests/ci/pvs_check.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index f68e5ca8210..a63b87d1d59 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -82,11 +82,16 @@ if __name__ == "__main__": os.makedirs(temp_path) new_repo_path = os.path.join(temp_path, repo_path) + logging.info("Will try to copy repo to %s", new_repo_path) if os.path.exists(new_repo_path): + logging.info("Removing old copy") shutil.rmtree(new_repo_path) + + logging.info("Copy repo from %s (exists %s) to %s", repo_path, os.path.exists(repo_path), temp_path) shutil.copytree(repo_path, temp_path) # this check modify repository so copy it to the temp directory repo_path = new_repo_path + logging.info("Repo copy path %s", repo_path) aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") From d510bfbe636c9160a370be4800d790f9570db5f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 12:59:31 +0300 Subject: [PATCH 151/950] Better --- .github/workflows/main.yml | 4 +++- tests/ci/pvs_check.py | 15 +-------------- 2 files changed, 4 insertions(+), 15 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 4e5c2ed19e9..2fb0e54a8ee 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -73,4 +73,6 @@ jobs: YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} PVS_STUDIO_KEY: ${{ secrets.PVS_STUDIO_KEY }} - run: cd $GITHUB_WORKSPACE/tests/ci && python3 pvs_check.py + TEMP_PATH: ${{runner.temp}}/pvs_check + REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse + run: mkdir -p ${{runner.temp}}/pvs_check && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 pvs_check.py diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index a63b87d1d59..c254ad74ae4 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -71,26 +71,13 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../"))) + repo_path = os.path.join(os.getenv("REPO_COPY", os.path.abspath("../../"))) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'pvs_check') with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: event = json.load(event_file) pr_info = PRInfo(event) - - if not os.path.exists(temp_path): - os.makedirs(temp_path) - - new_repo_path = os.path.join(temp_path, repo_path) - logging.info("Will try to copy repo to %s", new_repo_path) - if os.path.exists(new_repo_path): - logging.info("Removing old copy") - shutil.rmtree(new_repo_path) - - logging.info("Copy repo from %s (exists %s) to %s", repo_path, os.path.exists(repo_path), temp_path) - shutil.copytree(repo_path, temp_path) # this check modify repository so copy it to the temp directory - repo_path = new_repo_path logging.info("Repo copy path %s", repo_path) aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") From 68c7f8638ad82a1058f9dacde4cdb64583fd72ff Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 13:34:45 +0300 Subject: [PATCH 152/950] update docker image --- docker/test/pvs/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/pvs/Dockerfile b/docker/test/pvs/Dockerfile index f4675d35819..4eeb9855274 100644 --- a/docker/test/pvs/Dockerfile +++ b/docker/test/pvs/Dockerfile @@ -37,6 +37,8 @@ RUN set -x \ || echo "WARNING: Some file was just downloaded from the internet without any validation and we are installing it into the system"; } \ && dpkg -i "${PKG_VERSION}.deb" +ENV CCACHE_DIR=/test_ouput/ccache + CMD echo "Running PVS version $PKG_VERSION" && cd /repo_folder && pvs-studio-analyzer credentials $LICENCE_NAME $LICENCE_KEY -o ./licence.lic \ && cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF -DCMAKE_C_COMPILER=clang-13 -DCMAKE_CXX_COMPILER=clang\+\+-13 \ && ninja re2_st clickhouse_grpc_protos \ From 5d35ba7d9353e842e435aac372d389b91bc51c70 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 13:41:29 +0300 Subject: [PATCH 153/950] One more time --- docker/test/pvs/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/pvs/Dockerfile b/docker/test/pvs/Dockerfile index 4eeb9855274..06c2c424a74 100644 --- a/docker/test/pvs/Dockerfile +++ b/docker/test/pvs/Dockerfile @@ -39,7 +39,7 @@ RUN set -x \ ENV CCACHE_DIR=/test_ouput/ccache -CMD echo "Running PVS version $PKG_VERSION" && cd /repo_folder && pvs-studio-analyzer credentials $LICENCE_NAME $LICENCE_KEY -o ./licence.lic \ +CMD echo "Running PVS version $PKG_VERSION" && mkdir -p $CCACHE_DIR && cd /repo_folder && pvs-studio-analyzer credentials $LICENCE_NAME $LICENCE_KEY -o ./licence.lic \ && cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF -DCMAKE_C_COMPILER=clang-13 -DCMAKE_CXX_COMPILER=clang\+\+-13 \ && ninja re2_st clickhouse_grpc_protos \ && pvs-studio-analyzer analyze -o pvs-studio.log -e contrib -j 4 -l ./licence.lic; \ From 00d9a62d6cf400bc48c4514c9b1be69e7f3e719f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Sep 2021 14:17:50 +0300 Subject: [PATCH 154/950] Missclick --- docker/test/pvs/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/pvs/Dockerfile b/docker/test/pvs/Dockerfile index 06c2c424a74..77cbd910922 100644 --- a/docker/test/pvs/Dockerfile +++ b/docker/test/pvs/Dockerfile @@ -37,7 +37,7 @@ RUN set -x \ || echo "WARNING: Some file was just downloaded from the internet without any validation and we are installing it into the system"; } \ && dpkg -i "${PKG_VERSION}.deb" -ENV CCACHE_DIR=/test_ouput/ccache +ENV CCACHE_DIR=/test_output/ccache CMD echo "Running PVS version $PKG_VERSION" && mkdir -p $CCACHE_DIR && cd /repo_folder && pvs-studio-analyzer credentials $LICENCE_NAME $LICENCE_KEY -o ./licence.lic \ && cmake . -D"ENABLE_EMBEDDED_COMPILER"=OFF -D"USE_INTERNAL_PROTOBUF_LIBRARY"=OFF -D"USE_INTERNAL_GRPC_LIBRARY"=OFF -DCMAKE_C_COMPILER=clang-13 -DCMAKE_CXX_COMPILER=clang\+\+-13 \ From 5f97cad9a4e1abf200a46a4abc2b26d6a141656a Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 17 Sep 2021 15:32:01 +0000 Subject: [PATCH 155/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 38 +++++++++++++++---------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index eb95a094168..6d380c17f38 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -61,8 +61,9 @@ StorageFileLog::StorageFileLog( if (std::filesystem::is_regular_file(path)) { - file_status[path] = FileContext{}; - file_names.push_back(path); + auto normal_path = std::filesystem::path(path).lexically_normal().native(); + file_status[normal_path] = FileContext{}; + file_names.push_back(normal_path); } else if (std::filesystem::is_directory(path)) { @@ -72,8 +73,9 @@ StorageFileLog::StorageFileLog( { if (dir_entry.is_regular_file()) { - file_status[dir_entry.path()] = FileContext{}; - file_names.push_back(dir_entry.path()); + auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); + file_status[normal_path] = FileContext{}; + file_names.push_back(normal_path); } } } @@ -381,31 +383,37 @@ bool StorageFileLog::updateFileStatus() { switch (event.type) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: + case Poco::DirectoryWatcher::DW_ITEM_ADDED: { + auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path)) + if (std::filesystem::is_regular_file(normal_path)) { file_status[event.path] = FileContext{}; - file_names.push_back(event.path); + file_names.push_back(normal_path); } break; + } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { + auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); + if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) { - file_status[event.path].status = FileStatus::UPDATED; + file_status[normal_path].status = FileStatus::UPDATED; } break; + } case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { + auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); + if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) { - file_status[event.path].status = FileStatus::REMOVED; + file_status[normal_path].status = FileStatus::REMOVED; } + } } } std::vector valid_files; From 4fbc7c3f31ef97f4a017847336caeb58ff5c6e9f Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 18 Sep 2021 03:01:25 +0000 Subject: [PATCH 156/950] remove unused code --- src/Storages/FileLog/StorageFileLog.cpp | 4 ---- src/Storages/FileLog/StorageFileLog.h | 2 -- tests/queries/0_stateless/02023_storage_filelog.sh | 1 - 3 files changed, 7 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 6d380c17f38..10d838c84ec 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -307,10 +307,6 @@ bool StorageFileLog::streamToViews() return updateFileStatus(); } -void StorageFileLog::clearInvalidFiles() -{ -} - void registerStorageFileLog(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index d035c0e2903..7df66b5c54f 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -101,8 +101,6 @@ private: void threadFunc(); - void clearInvalidFiles(); - size_t getPollMaxBatchSize() const; size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 5a0333b1bc3..8b072dd65df 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -22,7 +22,6 @@ echo 8, 8 >> ${user_files_path}/logs/a.txt echo 9, 9 >> ${user_files_path}/logs/a.txt echo 10, 10 >> ${user_files_path}/logs/a.txt -### 1st TEST in CLIENT mode. ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" From 82a849ba8e1b95833d2ae7f3bcdd3e1c008bacb5 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Sat, 18 Sep 2021 09:36:02 +0000 Subject: [PATCH 157/950] add options method --- programs/server/config.xml | 19 ++++++++++++++++++ src/Interpreters/ClientInfo.h | 5 +++-- src/Server/HTTPHandler.cpp | 37 +++++++++++++++++++++++++++++++++++ 3 files changed, 59 insertions(+), 2 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 6c98ac740fe..bec51de6126 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -62,6 +62,25 @@ --> + +
+ Access-Control-Allow-Origin + * +
+
+ Access-Control-Allow-Headers + origin, x-requested-with +
+
+ Access-Control-Allow-Methods + POST, GET, OPTIONS +
+
+ Access-Control-Max-Age + 86400 +
+
+ diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 71570778645..294bf3b426c 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -35,8 +35,9 @@ public: enum class HTTPMethod : uint8_t { UNKNOWN = 0, - GET = 1, - POST = 2, + GET = 1, + POST = 2, + OPTIONS = 3 }; enum class QueryKind : uint8_t diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 0492b58dc88..99502261aa9 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -32,6 +32,7 @@ #include #include #include +#include "Server/HTTP/HTTPResponse.h" #if !defined(ARCADIA_BUILD) # include @@ -108,6 +109,37 @@ namespace ErrorCodes extern const int HTTP_LENGTH_REQUIRED; } +namespace +{ + /// Process options request. Usefull for CORS. + void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) + { + /// If answer for options request was not defined, return 501 to client. + if (!config.has("http_options_response")) + { + response.setStatusAndReason(HTTPResponse::HTTP_NOT_IMPLEMENTED); + response.send(); + } + else + { + /// otherwise fill response. + Strings config_keys; + config.keys("http_options_response", config_keys); + for (const std::string & config_key : config_keys) + { + if (config_key == "header" || config_key.starts_with("header[")) + { + response.add(config.getString("http_options_response." + config_key + ".name", "Empty header"), + config.getString("http_options_response." + config_key + ".value", "")); + response.setKeepAlive(false); + } + } + response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT); + response.send(); + } + } +} + static String base64Decode(const String & encoded) { String decoded; @@ -850,6 +882,11 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse try { + if (request.getMethod() == HTTPServerRequest::HTTP_OPTIONS) + { + processOptionsRequest(response, server.config()); + return; + } response.setContentType("text/plain; charset=UTF-8"); response.set("X-ClickHouse-Server-Display-Name", server_display_name); /// For keep-alive to work. From 6277747e8aa22b75a788b3e97372678b5c5df756 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 20 Sep 2021 05:37:18 +0000 Subject: [PATCH 158/950] First draft --- .../functions/other-functions.md | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 11cd522c622..af17954fec7 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2354,3 +2354,39 @@ Result: │ 1 │ └─────────┘ ``` + +## shardNum {#shard-num} + +Returns the number of a shard which executes the query for a distributed query. +If query is not distributed then *constant value* is returned. + +**Syntax** + +``` sql +shardNum() +``` + +**Returned value** + +- Shard number. + +Type: [UInt32](../../sql-reference/data-types/int-uint.md). + +## shardCount {#shard-count} + +Returns the total number of shards which execute a distributed query. +If query is not distributed then *constant value* is returned. + +**Syntax** + +``` sql +shardCount() +``` + +**Returned value** + +- Total number of shards. + +Type: [UInt32](../../sql-reference/data-types/int-uint.md). + + From e3fa2e36346f86a6bac909aa05ab3782aabc9b12 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 21 Sep 2021 16:11:35 +0000 Subject: [PATCH 159/950] fix fix update tests update tests --- .../FileLog/FileLogDirectoryWatcher.cpp | 21 +++-- .../FileLog/FileLogDirectoryWatcher.h | 16 +++- src/Storages/FileLog/FileLogSettings.h | 6 +- src/Storages/FileLog/FileLogSource.cpp | 4 +- .../FileLog/ReadBufferFromFileLog.cpp | 10 +- src/Storages/FileLog/StorageFileLog.cpp | 93 +++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 8 +- .../0_stateless/02023_storage_filelog.sh | 18 +++- 8 files changed, 111 insertions(+), 65 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index bc731500d61..b27b4719933 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -3,8 +3,11 @@ #include FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_shared(path)) + : path(path_), dw(std::make_shared(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { + /// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported on Linux. + /// On other platforms, a file rename or move operation will be reported via a + /// DW_ITEM_REMOVED and a DW_ITEM_ADDED event. The order of these two events is not defined. dw->itemAdded += Poco::delegate(this, &FileLogDirectoryWatcher::onItemAdded); dw->itemRemoved += Poco::delegate(this, &FileLogDirectoryWatcher::onItemRemoved); dw->itemModified += Poco::delegate(this, &FileLogDirectoryWatcher::onItemModified); @@ -12,7 +15,7 @@ FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) dw->itemMovedTo += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedTo); } -FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() +FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset() { std::lock_guard lock(mutex); Events res; @@ -20,9 +23,12 @@ FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() return res; } -bool FileLogDirectoryWatcher::hasError() const +FileLogDirectoryWatcher::Error FileLogDirectoryWatcher::getErrorAndReset() { - return error; + std::lock_guard lock(mutex); + Error old_error = error; + error = {}; + return old_error; } const std::string & FileLogDirectoryWatcher::getPath() const @@ -82,7 +88,10 @@ void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::Direct events.emplace_back(de); } -void FileLogDirectoryWatcher::onError(const Poco::Exception &) +void FileLogDirectoryWatcher::onError(const Poco::Exception & e) { - error = true; + std::lock_guard lock(mutex); + LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); + error.has_error = true; + error.error_msg = e.message(); } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 651d1d9db2d..d21d3d1e1ce 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,6 +4,8 @@ #include #include +#include + #include #include @@ -19,12 +21,18 @@ public: using Events = std::vector; + struct Error + { + bool has_error = false; + std::string error_msg = {}; + }; + explicit FileLogDirectoryWatcher(const std::string & path_); ~FileLogDirectoryWatcher() = default; - Events getEvents(); + Events getEventsAndReset(); - bool hasError() const; + Error getErrorAndReset(); const std::string & getPath() const; @@ -40,9 +48,11 @@ private: const std::string path; std::shared_ptr dw; + Poco::Logger * log; + std::mutex mutex; Events events; - bool error = false; + Error error; }; diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index cfea9d1e195..ba76b4d02da 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -12,10 +12,8 @@ class ASTStorage; #define FILELOG_RELATED_SETTINGS(M) \ /* default is stream_poll_timeout_ms */ \ M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \ - /* default is min(max_block_size, kafka_max_block_size)*/ \ - M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ - /* default is = max_insert_block_size / kafka_num_consumers */ \ - M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ + M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single filelog poll.", 0) \ + M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from filelog.", 0) \ M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0) #define LIST_OF_FILELOG_SETTINGS(M) \ diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 08a8c4b6868..c1feb721af0 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -121,9 +121,7 @@ Chunk FileLogSource::generate() { total_rows = total_rows + new_rows; } - - /// poll succeed, but parse failed - else + else /// poll succeed, but parse failed { ++failed_poll_attempts; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index bf9cd2d9fce..83ad412f202 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -40,7 +40,6 @@ void ReadBufferFromFileLog::cleanUnprocessed() bool ReadBufferFromFileLog::poll() { - if (hasMorePolledRecords()) { allowed = true; @@ -98,7 +97,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ size_t read_records_size = 0; const auto & file_names = storage.getFileNames(); - auto & file_status = storage.getFileStatus(); + auto & file_status = storage.getFileStatuses(); size_t files_per_stream = file_names.size() / max_streams_number; size_t start = stream_number * files_per_stream; @@ -113,19 +112,24 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto reader = std::ifstream(file_names[i]); reader.seekg(0, reader.end); + assert(reader.good()); + auto stream_end = reader.tellg(); + assert(reader.good()); reader.seekg(file.last_read_position); + assert(reader.good()); + Record record; while (read_records_size < need_records_size && reader.tellg() < stream_end) { - Record record; std::getline(reader, record); new_records.emplace_back(record); ++read_records_size; } file.last_read_position = reader.tellg(); + assert(reader.good()); if (reader.tellg() == stream_end) { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 10d838c84ec..fa638c2a884 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -38,6 +38,7 @@ namespace ErrorCodes namespace { const auto RESCHEDULE_MS = 500; + const auto BACKOFF_TRESHOLD = 32000; const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) } @@ -54,40 +55,48 @@ StorageFileLog::StorageFileLog( , path(getContext()->getUserFilesPath() + "/" + relative_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) + , milliseconds_to_wait(RESCHEDULE_MS) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - if (std::filesystem::is_regular_file(path)) + try { - auto normal_path = std::filesystem::path(path).lexically_normal().native(); - file_status[normal_path] = FileContext{}; - file_names.push_back(normal_path); - } - else if (std::filesystem::is_directory(path)) - { - path_is_directory = true; - /// Just consider file with depth 1 - for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + if (std::filesystem::is_regular_file(path)) { - if (dir_entry.is_regular_file()) + auto normal_path = std::filesystem::path(path).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } + else if (std::filesystem::is_directory(path)) + { + path_is_directory = true; + /// Just consider file with depth 1 + for (const auto & dir_entry : std::filesystem::directory_iterator{path}) { - auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); - file_status[normal_path] = FileContext{}; - file_names.push_back(normal_path); + if (dir_entry.is_regular_file()) + { + auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } } } + else + { + throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + } + + directory_watch = std::make_unique(path); + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + task = std::make_shared(std::move(thread)); } - else + catch (...) { - throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + tryLogCurrentException(__PRETTY_FUNCTION__); } - - directory_watch = std::make_unique(path); - - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - task = std::make_shared(std::move(thread)); } Pipe StorageFileLog::read( @@ -101,7 +110,7 @@ Pipe StorageFileLog::read( { std::lock_guard lock(status_mutex); - updateFileStatus(); + updateFileStatuses(); /// No files to parse if (file_names.empty()) @@ -155,7 +164,6 @@ size_t StorageFileLog::getPollMaxBatchSize() const { size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value : getContext()->getSettingsRef().max_block_size.value; - return std::min(batch_size, getMaxBlockSize()); } @@ -211,12 +219,17 @@ void StorageFileLog::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - auto file_status_no_update = streamToViews(); - if (file_status_no_update) + if (streamToViews()) { LOG_TRACE(log, "Stream stalled. Reschedule."); + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; break; } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts-start_time); @@ -225,7 +238,7 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } - updateFileStatus(); + updateFileStatuses(); } } } @@ -236,7 +249,7 @@ void StorageFileLog::threadFunc() // Wait for attached views if (!task->stream_cancelled) - task->holder->scheduleAfter(RESCHEDULE_MS); + task->holder->scheduleAfter(milliseconds_to_wait); } @@ -304,7 +317,7 @@ bool StorageFileLog::streamToViews() LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return updateFileStatus(); + return updateFileStatuses(); } void registerStorageFileLog(StorageFactory & factory) @@ -365,15 +378,15 @@ void registerStorageFileLog(StorageFactory & factory) }); } -bool StorageFileLog::updateFileStatus() +bool StorageFileLog::updateFileStatuses() { /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function - auto error = directory_watch->hasError(); - if (error) - LOG_INFO(log, "Error happened during watching directory {}.", directory_watch->getPath()); + auto error = directory_watch->getErrorAndReset(); + if (error.has_error) + LOG_ERROR(log, "Error happened during watching directory {}: {}", directory_watch->getPath(), error.error_msg); - auto events = directory_watch->getEvents(); + auto events = directory_watch->getEventsAndReset(); for (const auto & event : events) { @@ -384,7 +397,7 @@ bool StorageFileLog::updateFileStatus() LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(normal_path)) { - file_status[event.path] = FileContext{}; + file_statuses[event.path] = FileContext{}; file_names.push_back(normal_path); } break; @@ -393,9 +406,9 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) + if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) { - file_status[normal_path].status = FileStatus::UPDATED; + file_statuses[normal_path].status = FileStatus::UPDATED; } break; } @@ -405,9 +418,9 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) + if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) { - file_status[normal_path].status = FileStatus::REMOVED; + file_statuses[normal_path].status = FileStatus::REMOVED; } } } @@ -415,9 +428,9 @@ bool StorageFileLog::updateFileStatus() std::vector valid_files; for (const auto & file_name : file_names) { - if (file_status.at(file_name).status == FileStatus::REMOVED) + if (file_statuses.at(file_name).status == FileStatus::REMOVED) { - file_status.erase(file_name); + file_statuses.erase(file_name); } else { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 7df66b5c54f..ff94b0a548c 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -52,7 +52,7 @@ public: const auto & getFormatName() const { return format_name; } auto & getFileNames() { return file_names; } - auto & getFileStatus() { return file_status; } + auto & getFileStatuses() { return file_statuses; } protected: StorageFileLog( @@ -78,7 +78,7 @@ private: }; using NameToFile = std::unordered_map; - NameToFile file_status; + NameToFile file_statuses; std::vector file_names; @@ -86,6 +86,8 @@ private: std::unique_ptr directory_watch = nullptr; + uint64_t milliseconds_to_wait; + // Stream thread struct TaskContext { @@ -108,7 +110,7 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - bool updateFileStatus(); + bool updateFileStatuses(); }; } diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 8b072dd65df..d6a9d77f796 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -33,6 +33,9 @@ ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt +# touch does not change file content, no event +touch ${user_files_path}/logs/a.txt + ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" @@ -40,14 +43,23 @@ ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt +# MOVE_FILE_FROM and MOVE_FILE_TO monitored +mv ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt +rm ${user_files_path}/logs/e.txt -sleep 5 +for i in 1 2 3 4 5 +do + sleep 1 +done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" -echo 111, 111 >> ${user_files_path}/logs/a.txt +echo 111, 111 >> ${user_files_path}/logs/c.txt -sleep 5 +for i in 1 2 3 4 5 +do + sleep 1 +done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" From cd6f8078b5f6ee57f790646cddf339cfad4e88d7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 22 Sep 2021 02:07:59 +0000 Subject: [PATCH 160/950] refactor some code --- src/Storages/FileLog/StorageFileLog.cpp | 63 +++++++++++++------------ src/Storages/FileLog/StorageFileLog.h | 2 + 2 files changed, 36 insertions(+), 29 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index fa638c2a884..9711b8dbfc4 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -63,35 +63,7 @@ StorageFileLog::StorageFileLog( try { - if (std::filesystem::is_regular_file(path)) - { - auto normal_path = std::filesystem::path(path).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); - } - else if (std::filesystem::is_directory(path)) - { - path_is_directory = true; - /// Just consider file with depth 1 - for (const auto & dir_entry : std::filesystem::directory_iterator{path}) - { - if (dir_entry.is_regular_file()) - { - auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); - } - } - } - else - { - throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); - } - - directory_watch = std::make_unique(path); - - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - task = std::make_shared(std::move(thread)); + init(); } catch (...) { @@ -99,6 +71,39 @@ StorageFileLog::StorageFileLog( } } +void StorageFileLog::init() +{ + if (std::filesystem::is_regular_file(path)) + { + auto normal_path = std::filesystem::path(path).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } + else if (std::filesystem::is_directory(path)) + { + path_is_directory = true; + /// Just consider file with depth 1 + for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + { + if (dir_entry.is_regular_file()) + { + auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } + } + } + else + { + throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + } + + directory_watch = std::make_unique(path); + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + task = std::make_shared(std::move(thread)); +} + Pipe StorageFileLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ff94b0a548c..5825f3b5094 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -101,6 +101,8 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; + void init(); + void threadFunc(); size_t getPollMaxBatchSize() const; From f637f4f2d167c6d7a120c01b7c98043b1b078124 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 22 Sep 2021 02:52:34 +0000 Subject: [PATCH 161/950] remove unused variable update test update --- src/Storages/FileLog/StorageFileLog.cpp | 1 - src/Storages/FileLog/StorageFileLog.h | 1 - tests/queries/0_stateless/02023_storage_filelog.sh | 12 ++++++++---- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 9711b8dbfc4..0404442c1ee 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -81,7 +81,6 @@ void StorageFileLog::init() } else if (std::filesystem::is_directory(path)) { - path_is_directory = true; /// Just consider file with depth 1 for (const auto & dir_entry : std::filesystem::directory_iterator{path}) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 5825f3b5094..795180ba9eb 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -66,7 +66,6 @@ protected: private: std::unique_ptr filelog_settings; const String path; - bool path_is_directory = false; const String format_name; Poco::Logger * log; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index d6a9d77f796..908d77f5c26 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -47,18 +47,22 @@ cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt mv ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt rm ${user_files_path}/logs/e.txt -for i in 1 2 3 4 5 +# the max value of reschedule time is 32s, so 40s should +# always be enough to finish streamToViews +for _ in {1..400} do - sleep 1 + sleep 0.1 done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/c.txt -for i in 1 2 3 4 5 +# the max value of reschedule time is 32s, so 40s should +# always be enough to finish streamToViews +for _ in {1..400} do - sleep 1 + sleep 0.1 done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" From c8892ec7a71eac73a852ab1b8d200a86148b08c5 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Wed, 22 Sep 2021 10:34:48 +0000 Subject: [PATCH 162/950] add options support --- src/Server/HTTPHandler.cpp | 19 +++++++++++++------ src/Server/HTTPHandlerFactory.cpp | 4 ++-- src/Server/HTTPHandlerFactory.h | 6 ++++-- 3 files changed, 19 insertions(+), 10 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 99502261aa9..017bc82a475 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -25,6 +25,7 @@ #include #include #include +#include "common/logger_useful.h" #include #include #include @@ -111,10 +112,11 @@ namespace ErrorCodes namespace { - /// Process options request. Usefull for CORS. + /// Process options request. Useful for CORS. void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) { - /// If answer for options request was not defined, return 501 to client. + /// If response for options request was not defined, return 501 to client. + /// TODO should it be here? if (!config.has("http_options_response")) { response.setStatusAndReason(HTTPResponse::HTTP_NOT_IMPLEMENTED); @@ -129,12 +131,17 @@ namespace { if (config_key == "header" || config_key.starts_with("header[")) { - response.add(config.getString("http_options_response." + config_key + ".name", "Empty header"), - config.getString("http_options_response." + config_key + ".value", "")); - response.setKeepAlive(false); + /// If there is empty header name, it will not be processed and message about it will be in logs + if (config.getString("http_options_response." + config_key + ".name", "").empty()) + LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed."); + else + response.add(config.getString("http_options_response." + config_key + ".name", ""), + config.getString("http_options_response." + config_key + ".value", "")); + } } - response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT); + response.setKeepAlive(false); + response.setStatusAndReason(HTTPResponse::HTTP_OK); response.send(); } } diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index 1e3d02b85ab..526b86a5c28 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -123,7 +123,7 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I addCommonDefaultHandlersFactory(*factory, server); auto main_handler = std::make_shared>(server); - main_handler->allowPostAndGetParamsRequest(); + main_handler->allowPostAndGetParamsAndOptionsRequest(); factory->addHandler(main_handler); return factory; @@ -180,7 +180,7 @@ void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer addCommonDefaultHandlersFactory(factory, server); auto query_handler = std::make_shared>(server, "query"); - query_handler->allowPostAndGetParamsRequest(); + query_handler->allowPostAndGetParamsAndOptionsRequest(); factory.addHandler(query_handler); /// We check that prometheus handler will be served on current (default) port. diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index 6297f988eaa..5497d585d43 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -104,11 +104,13 @@ public: } /// Handle POST or GET with params - void allowPostAndGetParamsRequest() + void allowPostAndGetParamsAndOptionsRequest() { addFilter([](const auto & request) { - return request.getURI().find('?') != std::string::npos + return (request.getURI().find('?') != std::string::npos + && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST; }); } From 2cffa98a60677517a372f72a1fea746a19bb0328 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Wed, 22 Sep 2021 12:22:21 +0000 Subject: [PATCH 163/950] add test and comments in config --- programs/server/config.xml | 6 ++++-- src/Server/HTTPHandler.cpp | 14 ++++--------- tests/config/config.d/CORS.xml | 20 +++++++++++++++++++ .../02029_test_options_requests.reference | 5 +++++ .../02029_test_options_requests.sh | 8 ++++++++ 5 files changed, 41 insertions(+), 12 deletions(-) create mode 100644 tests/config/config.d/CORS.xml create mode 100644 tests/queries/0_stateless/02029_test_options_requests.reference create mode 100755 tests/queries/0_stateless/02029_test_options_requests.sh diff --git a/programs/server/config.xml b/programs/server/config.xml index bec51de6126..26c3107e972 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -62,7 +62,9 @@ --> - + + + random + + + 1 diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index a4fe3649e6f..5bc10841726 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -29,9 +29,6 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() { headers_started_sending = true; - if (add_cors_header) - response.set("Access-Control-Allow-Origin", "*"); - setResponseDefaultHeaders(response, keep_alive_timeout); if (!is_http_method_head) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index b4ff454195f..7cebf5ca770 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -36,7 +36,6 @@ private: HTTPServerResponse & response; bool is_http_method_head; - bool add_cors_header = false; unsigned keep_alive_timeout = 0; bool compress = false; CompressionMethod compression_method; @@ -103,13 +102,6 @@ public: compression_level = level; } - /// Turn CORS on or off. - /// The setting has any effect only if HTTP headers haven't been sent yet. - void addHeaderCORS(bool enable_cors) - { - add_cors_header = enable_cors; - } - /// Don't send HTTP headers with progress more frequently. void setSendProgressInterval(size_t send_progress_interval_ms_) { diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index cec7e1c8b3d..c27d5343e90 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -33,7 +33,7 @@ #include #include #include -#include "Server/HTTP/HTTPResponse.h" +#include #if !defined(ARCADIA_BUILD) # include @@ -112,33 +112,41 @@ namespace ErrorCodes namespace { - /// Process options request. Useful for CORS. - void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) +bool tryAddHeadersFromConfig(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) +{ + if (config.has("http_options_response")) { - /// If there is information for options request in cofing, fill response. - /// For this purpose find all headers related to http_options_response and add them with their values to response - if (config.has("http_options_response")) + Strings config_keys; + config.keys("http_options_response", config_keys); + for (const std::string & config_key : config_keys) { - Strings config_keys; - config.keys("http_options_response", config_keys); - for (const std::string & config_key : config_keys) + if (config_key == "header" || config_key.starts_with("header[")) { - if (config_key == "header" || config_key.starts_with("header[")) - { - /// If there is empty header name, it will not be processed and message about it will be in logs - if (config.getString("http_options_response." + config_key + ".name", "").empty()) - LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed."); - else - response.add(config.getString("http_options_response." + config_key + ".name", ""), - config.getString("http_options_response." + config_key + ".value", "")); + /// If there is empty header name, it will not be processed and message about it will be in logs + if (config.getString("http_options_response." + config_key + ".name", "").empty()) + LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed."); + else + response.add(config.getString("http_options_response." + config_key + ".name", ""), + config.getString("http_options_response." + config_key + ".value", "")); - } } - response.setKeepAlive(false); - response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT); - response.send(); } + return true; } + return false; +} + +/// Process options request. Useful for CORS. +void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config) +{ + /// If can add some headers from config + if (tryAddHeadersFromConfig(response, config)) + { + response.setKeepAlive(false); + response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT); + response.send(); + } +} } static String base64Decode(const String & encoded) @@ -739,9 +747,10 @@ void HTTPHandler::processQuery( if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress) static_cast(*in_post_maybe_compressed).disableChecksumming(); - /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed - /// Origin header. - used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); + /// Add CORS header if 'add_http_cors_header' setting is turned on or config has http_options_response, + /// which means that there are some headers to be sent, and the client passed Origin header. + if ((settings.add_http_cors_header || config.has("http_options_response")) && !request.get("Origin", "").empty()) + tryAddHeadersFromConfig(response, config); auto append_callback = [context] (ProgressCallback callback) { From ce4193fe957367d28da59e6c94fc54aefb3038db Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Wed, 22 Sep 2021 17:10:15 +0000 Subject: [PATCH 167/950] small refactoring --- src/Server/HTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c27d5343e90..046e7bdfaad 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -749,7 +749,7 @@ void HTTPHandler::processQuery( /// Add CORS header if 'add_http_cors_header' setting is turned on or config has http_options_response, /// which means that there are some headers to be sent, and the client passed Origin header. - if ((settings.add_http_cors_header || config.has("http_options_response")) && !request.get("Origin", "").empty()) + if (settings.add_http_cors_header && config.has("http_options_response") && !request.get("Origin", "").empty()) tryAddHeadersFromConfig(response, config); auto append_callback = [context] (ProgressCallback callback) From 9456c674864172c1125341a0421c92bd3c234ccb Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 22 Sep 2021 07:47:36 +0000 Subject: [PATCH 168/950] fix fix test fix fix fix exception handle --- .../FileLog/ReadBufferFromFileLog.cpp | 65 +++++++++++++++---- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.h | 3 +- .../0_stateless/02023_storage_filelog.sh | 7 +- 4 files changed, 60 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 83ad412f202..6d3a637a6bb 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -10,6 +10,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_FROM_ISTREAM; +} ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, @@ -97,7 +101,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ size_t read_records_size = 0; const auto & file_names = storage.getFileNames(); - auto & file_status = storage.getFileStatuses(); + auto & file_statuses = storage.getFileStatuses(); size_t files_per_stream = file_names.size() / max_streams_number; size_t start = stream_number * files_per_stream; @@ -105,40 +109,75 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ for (size_t i = start; i < end; ++i) { - auto & file = file_status[file_names[i]]; + auto & file = file_statuses[file_names[i]]; if (file.status == StorageFileLog::FileStatus::NO_CHANGE) continue; auto reader = std::ifstream(file_names[i]); + /// check if ifstream is good. For example, if the file deleted during streamToViews, + /// this will return false because file does not exist anymore. + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } + reader.seekg(0, reader.end); - assert(reader.good()); + /// Exception may happen in seekg and tellg, then badbit will be set + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } auto stream_end = reader.tellg(); - assert(reader.good()); + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } + + /// file may broken(for example truncate), mark this file to BROKEN, + /// should be removed in next updateFileStatuses call + if (file.last_read_position > static_cast(stream_end)) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } reader.seekg(file.last_read_position); - assert(reader.good()); + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } Record record; while (read_records_size < need_records_size && reader.tellg() < stream_end) { + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } std::getline(reader, record); new_records.emplace_back(record); ++read_records_size; } - file.last_read_position = reader.tellg(); - assert(reader.good()); + auto current_position = reader.tellg(); + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } - if (reader.tellg() == stream_end) + file.last_read_position = current_position; + + /// stream reach to end + if (current_position == stream_end && file.status != StorageFileLog::FileStatus::BROKEN) { file.status = StorageFileLog::FileStatus::NO_CHANGE; - /// All ifstream reach end - if (i == end - 1) - { - stream_out = true; - } + } + + /// All ifstream reach end or broken + if (i == end - 1 && (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::BEGIN)) + { + stream_out = true; } if (read_records_size == need_records_size) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 0404442c1ee..09a90fd7155 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -432,7 +432,7 @@ bool StorageFileLog::updateFileStatuses() std::vector valid_files; for (const auto & file_name : file_names) { - if (file_statuses.at(file_name).status == FileStatus::REMOVED) + if (file_statuses.at(file_name).status == FileStatus::REMOVED || file_statuses.at(file_name).status == FileStatus::BROKEN) { file_statuses.erase(file_name); } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 795180ba9eb..d149fb9c548 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -28,7 +28,8 @@ public: BEGIN, NO_CHANGE, UPDATED, - REMOVED + REMOVED, + BROKEN }; using Files = std::vector; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 908d77f5c26..97a147479ac 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: long, no-parallel + set -eu CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -49,7 +51,7 @@ rm ${user_files_path}/logs/e.txt # the max value of reschedule time is 32s, so 40s should # always be enough to finish streamToViews -for _ in {1..400} +for _ in {1..150} do sleep 0.1 done @@ -57,10 +59,11 @@ done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/c.txt +truncate ${user_files_path}/logs/d.txt --size 0 # the max value of reschedule time is 32s, so 40s should # always be enough to finish streamToViews -for _ in {1..400} +for _ in {1..150} do sleep 0.1 done From 7bbd08cb5d4c90357fc23b0cbfe96f36cfecff33 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Fri, 24 Sep 2021 15:40:27 +0300 Subject: [PATCH 169/950] Update HTTPHandler.cpp --- src/Server/HTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index bd4452ac6cb..7357c56ad2e 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -125,7 +125,7 @@ bool tryAddHeadersFromConfig(HTTPServerResponse & response, const Poco::Util::La /// If there is empty header name, it will not be processed and message about it will be in logs if (config.getString("http_options_response." + config_key + ".name", "").empty()) LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed."); - else + else response.add(config.getString("http_options_response." + config_key + ".name", ""), config.getString("http_options_response." + config_key + ".value", "")); From 0009cab96299eef8520424c2ac0b7e17229b972d Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 24 Sep 2021 16:44:22 +0000 Subject: [PATCH 170/950] add virtual columns --- src/Storages/FileLog/FileLogSource.cpp | 89 ++++--------------- src/Storages/FileLog/FileLogSource.h | 4 +- .../FileLog/ReadBufferFromFileLog.cpp | 9 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 11 ++- src/Storages/FileLog/StorageFileLog.cpp | 66 ++++++++++---- src/Storages/FileLog/StorageFileLog.h | 4 + 6 files changed, 91 insertions(+), 92 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index c1feb721af0..c272df1c2b2 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -21,20 +22,19 @@ FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, - const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockWithVirtuals(storage_.getVirtuals())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) - , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) , non_virtual_header(metadata_snapshot_->getSampleBlockNonMaterialized()) - , column_names_and_types(metadata_snapshot_->getColumns().getByNames(ColumnsDescription::All, columns, true)) + , virtual_header( + metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } @@ -44,53 +44,12 @@ Chunk FileLogSource::generate() if (!buffer || buffer->noRecords()) return {}; - MutableColumns read_columns = non_virtual_header.cloneEmptyColumns(); + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + auto input_format + = FormatFactory::instance().getInputFormat(storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); - InputPort port(input_format->getPort().getHeader(), input_format.get()); - connect(input_format->getPort(), port); - port.setNeeded(); - - std::optional exception_message; - auto read_file_log = [&] { - size_t new_rows = 0; - while (true) - { - auto status = input_format->prepare(); - - switch (status) - { - case IProcessor::Status::Ready: - input_format->work(); - break; - - case IProcessor::Status::Finished: - input_format->resetParser(); - return new_rows; - - case IProcessor::Status::PortFull: - { - auto chunk = port.pull(); - - auto chunk_rows = chunk.getNumRows(); - new_rows += chunk_rows; - - auto columns = chunk.detachColumns(); - for (size_t i = 0, s = columns.size(); i < s; ++i) - { - read_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - } - break; - } - case IProcessor::Status::NeedData: - case IProcessor::Status::Async: - case IProcessor::Status::ExpandPipeline: - throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); - } - } - }; + StreamingFormatExecutor executor(non_virtual_header, input_format); size_t total_rows = 0; size_t failed_poll_attempts = 0; @@ -100,25 +59,17 @@ Chunk FileLogSource::generate() { size_t new_rows = 0; if (buffer->poll()) - { - try - { - new_rows = read_file_log(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - else - { - /// No records polled, should break out early, since - /// file status can not be updated during streamToViews - break; - } + new_rows = executor.execute(); if (new_rows) { + auto file_name = buffer->getFileName(); + auto offset = buffer->getOffset(); + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(file_name); + virtual_columns[1]->insert(offset); + } total_rows = total_rows + new_rows; } else /// poll succeed, but parse failed @@ -137,13 +88,11 @@ Chunk FileLogSource::generate() if (total_rows == 0) return {}; - Columns result_columns; - result_columns.reserve(column_names_and_types.size()); + auto result_columns = executor.getResultColumns(); - for (const auto & elem : column_names_and_types) + for (auto & column : virtual_columns) { - auto index = non_virtual_header.getPositionByName(elem.getNameInStorage()); - result_columns.emplace_back(std::move(read_columns[index])); + result_columns.emplace_back(std::move(column)); } return Chunk(std::move(result_columns), total_rows); diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 1dc55f2c0c2..7f7d75bfaad 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -19,7 +19,6 @@ public: StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, - const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, @@ -36,7 +35,6 @@ private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; ContextPtr context; - Names column_names; UInt64 max_block_size; size_t poll_time_out; @@ -44,7 +42,7 @@ private: std::unique_ptr buffer; Block non_virtual_header; - const NamesAndTypesList column_names_and_types; + Block virtual_header; }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 6d3a637a6bb..ad23edac2d4 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -155,7 +155,10 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ { throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); } - std::getline(reader, record); + UInt64 start_offset = reader.tellg(); + std::getline(reader, record.data); + record.file_name = file_names[i]; + record.offset = start_offset; new_records.emplace_back(record); ++read_records_size; } @@ -192,8 +195,8 @@ bool ReadBufferFromFileLog::nextImpl() if (!allowed || !hasMorePolledRecords()) return false; - auto * new_position = const_cast(current->data()); - BufferBase::set(new_position, current->size(), 0); + auto * new_position = const_cast(current->data.data()); + BufferBase::set(new_position, current->data.size(), 0); allowed = false; ++current; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 66a8aeaa49f..747a5f316fe 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -37,6 +37,9 @@ public: bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } + auto getFileName() const { return current != records.begin() ? (current - 1)->file_name : ""; } + auto getOffset() const { return current != records.begin() ? (current - 1)->offset : 0; } + private: enum class BufferStatus { @@ -63,7 +66,13 @@ private: bool allowed = true; - using Record = std::string; + using RecordData = std::string; + struct Record + { + RecordData data; + std::string file_name; + UInt64 offset; + }; using Records = std::vector; Records records; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 09a90fd7155..04b76404f56 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -8,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -15,15 +18,10 @@ #include #include #include -#include #include #include -#include #include #include -#include - -#include namespace DB { @@ -39,7 +37,7 @@ namespace { const auto RESCHEDULE_MS = 500; const auto BACKOFF_TRESHOLD = 32000; - const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) + const auto MAX_THREAD_WORK_DURATION_MS = 60000; } StorageFileLog::StorageFileLog( @@ -134,28 +132,45 @@ Pipe StorageFileLog::read( *this, metadata_snapshot, modified_context, - column_names, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); } - return Pipe::unitePipes(std::move(pipes)); + auto pipe = Pipe::unitePipes(std::move(pipes)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); + + return pipe; } void StorageFileLog::startup() { - task->holder->activateAndSchedule(); + if (task) + { + task->holder->activateAndSchedule(); + } } void StorageFileLog::shutdown() { - task->stream_cancelled = true; + if (task) + { + task->stream_cancelled = true; - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + } } size_t StorageFileLog::getMaxBlockSize() const @@ -286,22 +301,31 @@ bool StorageFileLog::streamToViews() Pipes pipes; pipes.reserve(max_streams_number); - auto names = block_io.out->getHeader().getNames(); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { pipes.emplace_back(std::make_shared( *this, metadata_snapshot, new_context, - names, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); } + auto pipe = Pipe::unitePipes(std::move(pipes)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + block_io.out->getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); QueryPipeline pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); + pipeline.init(std::move(pipe)); assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); @@ -384,6 +408,8 @@ void registerStorageFileLog(StorageFactory & factory) bool StorageFileLog::updateFileStatuses() { + if (!directory_watch) + return false; /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function auto error = directory_watch->getErrorAndReset(); @@ -446,4 +472,14 @@ bool StorageFileLog::updateFileStatuses() return events.empty() || file_names.empty(); } + +NamesAndTypesList StorageFileLog::getVirtuals() const +{ + return NamesAndTypesList{{"_file_name", std::make_shared()}, {"_offset", std::make_shared()}}; +} + +Names StorageFileLog::getVirtualColumnNames() +{ + return {"_file_name", "_offset"}; +} } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index d149fb9c548..dc9485fccb2 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -55,6 +55,10 @@ public: auto & getFileNames() { return file_names; } auto & getFileStatuses() { return file_statuses; } + NamesAndTypesList getVirtuals() const override; + + static Names getVirtualColumnNames(); + protected: StorageFileLog( const StorageID & table_id_, From 7af96b12b16ddc8f42b479db379a99a17075a241 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 07:22:45 +0000 Subject: [PATCH 171/950] add meta file fix and update tests --- src/Common/ErrorCodes.cpp | 6 + .../FileLog/ReadBufferFromFileLog.cpp | 80 +--- src/Storages/FileLog/ReadBufferFromFileLog.h | 7 +- src/Storages/FileLog/StorageFileLog.cpp | 445 ++++++++++++++---- src/Storages/FileLog/StorageFileLog.h | 92 +++- .../02022_storage_filelog_one_file.reference | 41 ++ .../02022_storage_filelog_one_file.sh | 42 ++ .../02023_storage_filelog.reference | 123 ++++- .../0_stateless/02023_storage_filelog.sh | 62 +-- .../02024_storage_filelog_mv.reference | 121 +++++ .../0_stateless/02024_storage_filelog_mv.sh | 58 +++ ...2025_storage_filelog_virtual_col.reference | 144 ++++++ .../02025_storage_filelog_virtual_col.sh | 62 +++ 13 files changed, 1056 insertions(+), 227 deletions(-) create mode 100644 tests/queries/0_stateless/02022_storage_filelog_one_file.reference create mode 100755 tests/queries/0_stateless/02022_storage_filelog_one_file.sh create mode 100644 tests/queries/0_stateless/02024_storage_filelog_mv.reference create mode 100755 tests/queries/0_stateless/02024_storage_filelog_mv.sh create mode 100644 tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference create mode 100755 tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 53276f5b196..9312fde8baa 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -588,6 +588,12 @@ M(618, LZ4_DECODER_FAILED) \ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ + M(621, PATH_DOESNT_EXIST) \ + M(622, FILE_META_NOT_FOUND) \ + M(623, CANNOT_GET_FILE_STAT) \ + M(624, NOT_REGULAR_FILE) \ + M(625, READ_META_FILE_FAILED) \ + M(626, FILE_STREAM_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index ad23edac2d4..bfcfb3a2385 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -69,8 +69,6 @@ bool ReadBufferFromFileLog::poll() allowed = true; return true; } - - } ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_size_) @@ -100,85 +98,52 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ size_t need_records_size = batch_size_ - new_records.size(); size_t read_records_size = 0; - const auto & file_names = storage.getFileNames(); - auto & file_statuses = storage.getFileStatuses(); + auto & file_infos = storage.getFileInfos(); - size_t files_per_stream = file_names.size() / max_streams_number; + size_t files_per_stream = file_infos.file_names.size() / max_streams_number; size_t start = stream_number * files_per_stream; - size_t end = stream_number == max_streams_number - 1 ? file_names.size() : (stream_number + 1) * files_per_stream; + size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; for (size_t i = start; i < end; ++i) { - auto & file = file_statuses[file_names[i]]; - if (file.status == StorageFileLog::FileStatus::NO_CHANGE) + auto file_name = file_infos.file_names[i]; + auto & file_ctx = file_infos.context_by_name.at(file_name); + if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) continue; - auto reader = std::ifstream(file_names[i]); - - /// check if ifstream is good. For example, if the file deleted during streamToViews, - /// this will return false because file does not exist anymore. - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - reader.seekg(0, reader.end); - /// Exception may happen in seekg and tellg, then badbit will be set - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - auto stream_end = reader.tellg(); - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - /// file may broken(for example truncate), mark this file to BROKEN, - /// should be removed in next updateFileStatuses call - if (file.last_read_position > static_cast(stream_end)) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - reader.seekg(file.last_read_position); - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } + auto & file_meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file_name)); Record record; - while (read_records_size < need_records_size && reader.tellg() < stream_end) + while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) { - if (!reader.good()) + if (!file_ctx.reader.good()) { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); } - UInt64 start_offset = reader.tellg(); - std::getline(reader, record.data); - record.file_name = file_names[i]; + UInt64 start_offset = file_ctx.reader.tellg(); + std::getline(file_ctx.reader, record.data); + record.file_name = file_name; record.offset = start_offset; new_records.emplace_back(record); ++read_records_size; } - auto current_position = reader.tellg(); - if (!reader.good()) + UInt64 current_position = file_ctx.reader.tellg(); + if (!file_ctx.reader.good()) { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); } - file.last_read_position = current_position; + file_meta.last_writen_position = current_position; /// stream reach to end - if (current_position == stream_end && file.status != StorageFileLog::FileStatus::BROKEN) + if (current_position == file_meta.last_open_end) { - file.status = StorageFileLog::FileStatus::NO_CHANGE; + file_ctx.status = StorageFileLog::FileStatus::NO_CHANGE; } - /// All ifstream reach end or broken - if (i == end - 1 && (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::BEGIN)) + /// All ifstream reach end + if (i == end - 1 && (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)) { stream_out = true; } @@ -199,6 +164,9 @@ bool ReadBufferFromFileLog::nextImpl() BufferBase::set(new_position, current->data.size(), 0); allowed = false; + current_file = current->file_name; + current_offset = current->offset; + ++current; return true; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 747a5f316fe..47a2770550d 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -37,8 +37,8 @@ public: bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } - auto getFileName() const { return current != records.begin() ? (current - 1)->file_name : ""; } - auto getOffset() const { return current != records.begin() ? (current - 1)->offset : 0; } + auto getFileName() const { return current_file; } + auto getOffset() const { return current_offset; } private: enum class BufferStatus @@ -78,6 +78,9 @@ private: Records records; Records::const_iterator current; + String current_file; + UInt64 current_offset; + using TaskThread = BackgroundSchedulePool::TaskHolder; TaskThread wait_task; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 04b76404f56..9b64be4c99b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,5 +1,9 @@ #include #include +#include +#include +#include +#include #include #include #include @@ -9,13 +13,13 @@ #include #include #include -#include #include #include #include #include #include #include +#include #include #include #include @@ -23,6 +27,8 @@ #include #include +#include + namespace DB { @@ -31,6 +37,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; + extern const int CANNOT_GET_FILE_STAT; + extern const int NOT_REGULAR_FILE; + extern const int READ_META_FILE_FAILED; + extern const int FILE_STREAM_ERROR; } namespace @@ -46,7 +56,8 @@ StorageFileLog::StorageFileLog( const ColumnsDescription & columns_, const String & relative_path_, const String & format_name_, - std::unique_ptr settings) + std::unique_ptr settings, + bool attach) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) @@ -61,7 +72,18 @@ StorageFileLog::StorageFileLog( try { - init(); + loadMetaFiles(attach); + loadFiles(); + + assert( + file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() + == file_infos.context_by_name.size()); + + if (path_is_directory) + directory_watch = std::make_unique(path); + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + task = std::make_shared(std::move(thread)); } catch (...) { @@ -69,36 +91,184 @@ StorageFileLog::StorageFileLog( } } -void StorageFileLog::init() +void StorageFileLog::loadMetaFiles(bool attach) { + const auto database = DatabaseCatalog::instance().getDatabase(getStorageID().getDatabaseName()); + const auto table_name = getStorageID().getTableName(); + + root_meta_path = database->getMetadataPath() + "/." + table_name; + + /// Create table, just create meta data directory + if (!attach) + { + if (std::filesystem::exists(root_meta_path)) + { + std::filesystem::remove_all(root_meta_path); + } + std::filesystem::create_directories(root_meta_path); + } + /// Attach table + else + { + /// Meta file may lost, log and create directory + if (!std::filesystem::exists(root_meta_path)) + { + LOG_INFO(log, "Meta files of table {} may have lost.", getStorageID().getTableName()); + std::filesystem::create_directories(root_meta_path); + } + /// Load all meta info to file_infos; + deserialize(); + } +} + +void StorageFileLog::loadFiles() +{ + if (std::filesystem::is_regular_file(path)) { - auto normal_path = std::filesystem::path(path).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); + path_is_directory = false; + root_data_path = getContext()->getUserFilesPath(); + + file_infos.file_names.push_back(std::filesystem::path(path).filename()); } else if (std::filesystem::is_directory(path)) { + root_data_path = path; /// Just consider file with depth 1 for (const auto & dir_entry : std::filesystem::directory_iterator{path}) { if (dir_entry.is_regular_file()) { - auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); + file_infos.file_names.push_back(dir_entry.path().filename()); } } } else { - throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", path); } - directory_watch = std::make_unique(path); + /// Get files inode + for (const auto & file : file_infos.file_names) + { + auto inode = getInode(getFullDataPath(file)); + file_infos.inode_by_name.emplace(file, inode); + file_infos.context_by_name.emplace(file, FileContext{}); + } - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - task = std::make_shared(std::move(thread)); + /// Update file meta or create file meta + for (const auto & file_inode : file_infos.inode_by_name) + { + if (auto it = file_infos.meta_by_inode.find(file_inode.second); it != file_infos.meta_by_inode.end()) + { + /// data file have been renamed, need update meta file's name + if (it->second.file_name != file_inode.first) + { + it->second.file_name = file_inode.first; + if (std::filesystem::exists(getFullMetaPath(it->second.file_name))) + { + std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file_inode.first)); + } + } + } + /// New file + else + { + FileMeta meta{file_inode.first, 0, 0}; + file_infos.meta_by_inode.emplace(file_inode.second, meta); + } + } + + /// Clear unneeded meta file, because data files may be deleted + if (file_infos.meta_by_inode.size() > file_infos.inode_by_name.size()) + { + InodeToFileMeta valid_metas; + valid_metas.reserve(file_infos.inode_by_name.size()); + for (const auto & it : file_infos.meta_by_inode) + { + if (file_infos.inode_by_name.contains(it.second.file_name)) + valid_metas.emplace(it); + } + file_infos.meta_by_inode.swap(valid_metas); + } +} + +void StorageFileLog::serialize(bool with_end_pos) const +{ + for (const auto & it : file_infos.meta_by_inode) + { + auto full_name = getFullMetaPath(it.second.file_name); + if (!std::filesystem::exists(full_name)) + { + Poco::File{full_name}.createFile(); + } + WriteBufferFromFile buf(full_name); + writeIntText(it.first, buf); + writeChar('\n', buf); + writeIntText(it.second.last_writen_position, buf); + + if (with_end_pos) + { + writeChar('\n', buf); + writeIntText(it.second.last_open_end, buf); + } + } +} + +void StorageFileLog::deserialize() +{ + for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) + { + if (!dir_entry.is_regular_file()) + { + throw Exception( + ErrorCodes::NOT_REGULAR_FILE, + "The file {} under {} is not a regular file when deserializing meta files.", + dir_entry.path().c_str(), + root_meta_path); + } + + ReadBufferFromFile buf(dir_entry.path().c_str()); + FileMeta meta; + UInt64 inode, last_written_pos; + + if (!tryReadIntText(inode, buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + if (!checkChar('\n', buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + if (!tryReadIntText(last_written_pos, buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + + meta.file_name = dir_entry.path().filename(); + meta.last_writen_position = last_written_pos; + + /// May have last open end in meta file + if (checkChar('\n', buf)) + { + if (!tryReadIntText(meta.last_open_end, buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + } + + file_infos.meta_by_inode.emplace(inode, meta); + } +} + +UInt64 StorageFileLog::getInode(const String & file_name) +{ + struct stat file_stat; + if (stat(file_name.c_str(), &file_stat)) + { + throw Exception(ErrorCodes::CANNOT_GET_FILE_STAT, "Can not get stat info of file {}", file_name); + } + return file_stat.st_ino; } Pipe StorageFileLog::read( @@ -112,45 +282,47 @@ Pipe StorageFileLog::read( { std::lock_guard lock(status_mutex); - updateFileStatuses(); + updateFileInfos(); /// No files to parse - if (file_names.empty()) + if (file_infos.file_names.empty()) { return Pipe{}; } auto modified_context = Context::createCopy(local_context); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); + + openFilesAndSetPos(); + serialize(true); Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - pipes.emplace_back(std::make_shared( - *this, - metadata_snapshot, - modified_context, - getMaxBlockSize(), - getPollTimeoutMillisecond(), - stream_number, - max_streams_number)); + Pipe pipe(std::make_shared( + *this, metadata_snapshot, modified_context, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); + pipes.emplace_back(std::move(pipe)); } - auto pipe = Pipe::unitePipes(std::move(pipes)); + return Pipe::unitePipes(std::move(pipes)); +} - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); - - return pipe; +void StorageFileLog::drop() +{ + if (std::filesystem::exists(root_meta_path)) + std::filesystem::remove_all(root_meta_path); } void StorageFileLog::startup() @@ -161,7 +333,6 @@ void StorageFileLog::startup() } } - void StorageFileLog::shutdown() { if (task) @@ -171,6 +342,59 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } + closeFilesAndStoreMeta(); +} + +void StorageFileLog::openFilesAndSetPos() +{ + for (const auto & file : file_infos.file_names) + { + auto & context = file_infos.context_by_name.at(file); + if (context.status != FileStatus::NO_CHANGE) + { + context.reader = std::ifstream(getFullDataPath(file)); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); + } + + context.reader.seekg(0, context.reader.end); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); + } + + auto file_end = context.reader.tellg(); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file); + } + + auto & meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file)); + if (meta.last_writen_position > static_cast(file_end)) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); + } + /// updte file end at the monment, used in ReadBuffer and serialize + meta.last_open_end = file_end; + + context.reader.seekg(meta.last_writen_position); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); + } + } + } +} + +void StorageFileLog::closeFilesAndStoreMeta() +{ + for (auto & it : file_infos.context_by_name) + { + if (it.second.reader.is_open()) + it.second.reader.close(); + } + serialize(); } size_t StorageFileLog::getMaxBlockSize() const @@ -192,7 +416,6 @@ size_t StorageFileLog::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } - bool StorageFileLog::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached @@ -257,7 +480,7 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } - updateFileStatuses(); + updateFileInfos(); } } } @@ -271,7 +494,6 @@ void StorageFileLog::threadFunc() task->holder->scheduleAfter(milliseconds_to_wait); } - bool StorageFileLog::streamToViews() { std::lock_guard lock(status_mutex); @@ -283,7 +505,7 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_infos.file_names.size()); /// No files to parse if (max_streams_number == 0) { @@ -299,33 +521,30 @@ bool StorageFileLog::streamToViews() InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); + openFilesAndSetPos(); + serialize(); + Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - pipes.emplace_back(std::make_shared( - *this, - metadata_snapshot, - new_context, - getPollMaxBatchSize(), - getPollTimeoutMillisecond(), - stream_number, - max_streams_number)); + Pipe pipe(std::make_shared( + *this, metadata_snapshot, new_context, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + block_io.out->getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); + pipes.emplace_back(std::move(pipe)); } - auto pipe = Pipe::unitePipes(std::move(pipes)); - - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - block_io.out->getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); @@ -338,14 +557,18 @@ bool StorageFileLog::streamToViews() { block_io.out->write(block); rows += block.rows(); + /// During files open, also save file end at the openning moment + serialize(true); } block_io.out->writeSuffix(); + serialize(); + UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return updateFileStatuses(); + return updateFileInfos(); } void registerStorageFileLog(StorageFactory & factory) @@ -395,7 +618,8 @@ void registerStorageFileLog(StorageFactory & factory) auto path = path_ast->as().value.safeGet(); auto format = format_ast->as().value.safeGet(); - return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings)); + return StorageFileLog::create( + args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings), args.attach); }; factory.registerStorage( @@ -406,10 +630,24 @@ void registerStorageFileLog(StorageFactory & factory) }); } -bool StorageFileLog::updateFileStatuses() +bool StorageFileLog::updateFileInfos() { if (!directory_watch) + { + /// For table just watch one file, we can not use directory monitor to watch it + if (!path_is_directory) + { + assert( + file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() + == file_infos.context_by_name.size() == 1); + if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::UPDATED; + return true; + } + } return false; + } /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function auto error = directory_watch->getErrorAndReset(); @@ -423,54 +661,95 @@ bool StorageFileLog::updateFileStatuses() switch (event.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: { - auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(normal_path)) + if (std::filesystem::is_regular_file(event.path)) { - file_statuses[event.path] = FileContext{}; - file_names.push_back(normal_path); + auto file_name = std::filesystem::path(event.path).filename(); + auto inode = getInode(event.path); + + file_infos.file_names.push_back(file_name); + file_infos.inode_by_name.emplace(file_name, inode); + + FileMeta meta{file_name, 0, 0}; + file_infos.meta_by_inode.emplace(inode, meta); + file_infos.context_by_name.emplace(file_name, FileContext{}); } break; } case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { - auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) + auto file_name = std::filesystem::path(event.path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { - file_statuses[normal_path].status = FileStatus::UPDATED; + it->second.status = FileStatus::UPDATED; } break; } case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { - auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) + auto file_name = std::filesystem::path(event.path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { - file_statuses[normal_path].status = FileStatus::REMOVED; + it->second.status = FileStatus::REMOVED; + } + break; + } + /// file rename + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { + auto file_name = std::filesystem::path(event.path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + + file_infos.file_names.push_back(file_name); + file_infos.context_by_name.emplace(file_name, FileContext{}); + + auto inode = getInode(event.path); + file_infos.inode_by_name.emplace(file_name, inode); + + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) + { + // rename meta file + auto old_name = it->second.file_name; + it->second.file_name = file_name; + if (std::filesystem::exists(getFullMetaPath(old_name))) + { + std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + } } } } } std::vector valid_files; - for (const auto & file_name : file_names) + + for (const auto & file_name : file_infos.file_names) { - if (file_statuses.at(file_name).status == FileStatus::REMOVED || file_statuses.at(file_name).status == FileStatus::BROKEN) + if (auto it = file_infos.context_by_name.find(file_name); + it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED) { - file_statuses.erase(file_name); + file_infos.context_by_name.erase(it); + if (auto inode = file_infos.inode_by_name.find(file_name); inode != file_infos.inode_by_name.end()) + { + file_infos.inode_by_name.erase(inode); + file_infos.meta_by_inode.erase(inode->second); + if (std::filesystem::exists(getFullMetaPath(file_name))) + std::filesystem::remove(getFullMetaPath(file_name)); + } } else { valid_files.push_back(file_name); } } + file_infos.file_names.swap(valid_files); - file_names.swap(valid_files); + /// These file infos should always have same size(one for one) + assert( + file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() + == file_infos.context_by_name.size()); - return events.empty() || file_names.empty(); + return events.empty() || file_infos.file_names.empty(); } NamesAndTypesList StorageFileLog::getVirtuals() const diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index dc9485fccb2..d715fbbfefc 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -23,14 +23,6 @@ class StorageFileLog final : public shared_ptr_helper, public IS friend struct shared_ptr_helper; public: - enum class FileStatus - { - BEGIN, - NO_CHANGE, - UPDATED, - REMOVED, - BROKEN - }; using Files = std::vector; @@ -50,15 +42,55 @@ public: size_t max_block_size, unsigned num_streams) override; + void drop() override; + const auto & getFormatName() const { return format_name; } - auto & getFileNames() { return file_names; } - auto & getFileStatuses() { return file_statuses; } + enum class FileStatus + { + OPEN, /// first time open file after table start up + NO_CHANGE, + UPDATED, + REMOVED, + }; + + struct FileContext + { + FileStatus status = FileStatus::OPEN; + std::ifstream reader; + }; + + struct FileMeta + { + String file_name; + UInt64 last_writen_position = 0; + UInt64 last_open_end; + }; + + using FileNameToInode = std::unordered_map; + using InodeToFileMeta = std::unordered_map; + using FileNameToContext = std::unordered_map; + + struct FileInfos + { + FileNameToInode inode_by_name; + InodeToFileMeta meta_by_inode; + FileNameToContext context_by_name; + /// file names without path + Names file_names; + }; + + auto & getFileInfos() { return file_infos; } + + auto getFullMetaPath(const String & file_name) const { return root_meta_path + "/" + file_name; } + auto getFullDataPath(const String & file_name) const { return root_data_path + "/" + file_name; } NamesAndTypesList getVirtuals() const override; static Names getVirtualColumnNames(); + static UInt64 getInode(const String & file_name); + protected: StorageFileLog( const StorageID & table_id_, @@ -66,33 +98,33 @@ protected: const ColumnsDescription & columns_, const String & relative_path_, const String & format_name_, - std::unique_ptr settings); + std::unique_ptr settings, + bool attach); private: std::unique_ptr filelog_settings; + + /// user_files_path/ + path_argument/ const String path; + bool path_is_directory = true; + + /// If path argument of the table is a regular file, it equals to user_files_path + /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path + String root_data_path; + /// Database meta_path/ + .table_name/ + String root_meta_path; + + FileInfos file_infos; const String format_name; Poco::Logger * log; - struct FileContext - { - FileStatus status = FileStatus::BEGIN; - size_t last_read_position = 0; - }; - - using NameToFile = std::unordered_map; - NameToFile file_statuses; - - std::vector file_names; - std::mutex status_mutex; std::unique_ptr directory_watch = nullptr; uint64_t milliseconds_to_wait; - // Stream thread struct TaskContext { BackgroundSchedulePool::TaskHolder holder; @@ -105,7 +137,9 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; - void init(); + void loadFiles(); + + void loadMetaFiles(bool attach); void threadFunc(); @@ -116,7 +150,15 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - bool updateFileStatuses(); + bool updateFileInfos(); + + void openFilesAndSetPos(); + void closeFilesAndStoreMeta(); + + /// Serialize all file meta + void serialize(bool with_end_pos = false) const; + + void deserialize(); }; } diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.reference b/tests/queries/0_stateless/02022_storage_filelog_one_file.reference new file mode 100644 index 00000000000..88332de037f --- /dev/null +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.reference @@ -0,0 +1,41 @@ +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 +110 110 +111 111 +112 112 +113 113 +114 114 +115 115 +116 116 +117 117 +118 118 +119 119 +120 120 diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh new file mode 100755 index 00000000000..63450a83311 --- /dev/null +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/a.txt +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('a.txt', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/a.txt +done + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +# touch does not change file content, no event +touch ${user_files_path}/a.txt +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +${CLICKHOUSE_CLIENT} --query "attach table file_log;" + +# should no records return +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +rm -rf ${user_files_path}/a.txt diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference index f1aa3abf257..0ab71c65c6b 100644 --- a/tests/queries/0_stateless/02023_storage_filelog.reference +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -8,7 +8,16 @@ 8 8 9 9 10 10 -100 100 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 1 1 2 2 3 3 @@ -19,7 +28,16 @@ 8 8 9 9 10 10 -100 100 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 1 1 1 1 2 2 @@ -40,28 +58,87 @@ 9 9 10 10 10 10 +11 11 +11 11 +12 12 +12 12 +13 13 +13 13 +14 14 +14 14 +15 15 +15 15 +16 16 +16 16 +17 17 +17 17 +18 18 +18 18 +19 19 +19 19 +20 20 +20 20 100 100 100 100 -1 1 -1 1 -2 2 -2 2 -3 3 -3 3 -4 4 -4 4 -5 5 -5 5 -6 6 -6 6 -7 7 -7 7 -8 8 -8 8 -9 9 -9 9 -10 10 -10 10 -100 100 100 100 +101 101 +101 101 +101 101 +102 102 +102 102 +102 102 +103 103 +103 103 +103 103 +104 104 +104 104 +104 104 +105 105 +105 105 +105 105 +106 106 +106 106 +106 106 +107 107 +107 107 +107 107 +108 108 +108 108 +108 108 +109 109 +109 109 +109 109 +110 110 +110 110 +110 110 111 111 +111 111 +111 111 +112 112 +112 112 +112 112 +113 113 +113 113 +113 113 +114 114 +114 114 +114 114 +115 115 +115 115 +115 115 +116 116 +116 116 +116 116 +117 117 +117 117 +117 117 +118 118 +118 118 +118 118 +119 119 +119 119 +119 119 +120 120 +120 120 +120 120 +OK diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 97a147479ac..eae9821d950 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,67 +10,53 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p ${user_files_path}/logs/ -echo 1, 1 >> ${user_files_path}/logs/a.txt -echo 2, 2 >> ${user_files_path}/logs/a.txt -echo 3, 3 >> ${user_files_path}/logs/a.txt -echo 4, 4 >> ${user_files_path}/logs/a.txt -echo 5, 5 >> ${user_files_path}/logs/a.txt -echo 6, 6 >> ${user_files_path}/logs/a.txt -echo 7, 7 >> ${user_files_path}/logs/a.txt -echo 8, 8 >> ${user_files_path}/logs/a.txt -echo 9, 9 >> ${user_files_path}/logs/a.txt -echo 10, 10 >> ${user_files_path}/logs/a.txt + +rm -rf ${user_files_path}/logs/* + +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -echo 100, 100 >> ${user_files_path}/logs/a.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done # touch does not change file content, no event touch ${user_files_path}/logs/a.txt -${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" - -${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" -${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" - cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt -# MOVE_FILE_FROM and MOVE_FILE_TO monitored -mv ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt -rm ${user_files_path}/logs/e.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt -# the max value of reschedule time is 32s, so 40s should -# always be enough to finish streamToViews -for _ in {1..150} -do - sleep 0.1 -done +rm ${user_files_path}/logs/d.txt -${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -echo 111, 111 >> ${user_files_path}/logs/c.txt -truncate ${user_files_path}/logs/d.txt --size 0 +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +${CLICKHOUSE_CLIENT} --query "attach table file_log;" -# the max value of reschedule time is 32s, so 40s should -# always be enough to finish streamToViews -for _ in {1..150} -do - sleep 0.1 -done +# should no records return +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" +truncate ${user_files_path}/logs/a.txt --size 0 + +# exception happend +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -${CLICKHOUSE_CLIENT} --query "drop table mv;" rm -rf ${user_files_path}/logs diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.reference b/tests/queries/0_stateless/02024_storage_filelog_mv.reference new file mode 100644 index 00000000000..2ebb200a43a --- /dev/null +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.reference @@ -0,0 +1,121 @@ +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 +2 2 +3 3 +3 3 +3 3 +3 3 +4 4 +4 4 +4 4 +4 4 +5 5 +5 5 +5 5 +5 5 +6 6 +6 6 +6 6 +6 6 +7 7 +7 7 +7 7 +7 7 +8 8 +8 8 +8 8 +8 8 +9 9 +9 9 +9 9 +9 9 +10 10 +10 10 +10 10 +10 10 +11 11 +11 11 +11 11 +11 11 +12 12 +12 12 +12 12 +12 12 +13 13 +13 13 +13 13 +13 13 +14 14 +14 14 +14 14 +14 14 +15 15 +15 15 +15 15 +15 15 +16 16 +16 16 +16 16 +16 16 +17 17 +17 17 +17 17 +17 17 +18 18 +18 18 +18 18 +18 18 +19 19 +19 19 +19 19 +19 19 +20 20 +20 20 +20 20 +20 20 +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 +110 110 +111 111 +112 112 +113 113 +114 114 +115 115 +116 116 +117 117 +118 118 +119 119 +120 120 diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh new file mode 100755 index 00000000000..d6d912624a8 --- /dev/null +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +mkdir -p ${user_files_path}/logs/ +rm -rf ${user_files_path}/logs/* +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" +${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" + +for i in {1..200} +do + sleep 0.1 +done + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt + +# touch does not change file content, no event +touch ${user_files_path}/logs/a.txt + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt + +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/logs/d.txt +done + +for _ in {1..300} +do + sleep 0.1 +done + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +${CLICKHOUSE_CLIENT} --query "drop table mv;" +${CLICKHOUSE_CLIENT} --query "drop table file_log;" + +rm -rf ${user_files_path}/logs diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference new file mode 100644 index 00000000000..aaa25ebb451 --- /dev/null +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference @@ -0,0 +1,144 @@ +1 1 a.txt 0 +2 2 a.txt 5 +3 3 a.txt 10 +4 4 a.txt 15 +5 5 a.txt 20 +6 6 a.txt 25 +7 7 a.txt 30 +8 8 a.txt 35 +9 9 a.txt 40 +10 10 a.txt 45 +11 11 a.txt 52 +12 12 a.txt 59 +13 13 a.txt 66 +14 14 a.txt 73 +15 15 a.txt 80 +16 16 a.txt 87 +17 17 a.txt 94 +18 18 a.txt 101 +19 19 a.txt 108 +20 20 a.txt 115 +1 1 b.txt 0 +2 2 b.txt 5 +3 3 b.txt 10 +4 4 b.txt 15 +5 5 b.txt 20 +6 6 b.txt 25 +7 7 b.txt 30 +8 8 b.txt 35 +9 9 b.txt 40 +10 10 b.txt 45 +11 11 b.txt 52 +12 12 b.txt 59 +13 13 b.txt 66 +14 14 b.txt 73 +15 15 b.txt 80 +16 16 b.txt 87 +17 17 b.txt 94 +18 18 b.txt 101 +19 19 b.txt 108 +20 20 b.txt 115 +100 100 a.txt 122 +101 101 a.txt 131 +102 102 a.txt 140 +103 103 a.txt 149 +104 104 a.txt 158 +105 105 a.txt 167 +106 106 a.txt 176 +107 107 a.txt 185 +108 108 a.txt 194 +109 109 a.txt 203 +110 110 a.txt 212 +111 111 a.txt 221 +112 112 a.txt 230 +113 113 a.txt 239 +114 114 a.txt 248 +115 115 a.txt 257 +116 116 a.txt 266 +117 117 a.txt 275 +118 118 a.txt 284 +119 119 a.txt 293 +120 120 a.txt 302 +1 1 c.txt 0 +2 2 c.txt 5 +3 3 c.txt 10 +4 4 c.txt 15 +5 5 c.txt 20 +6 6 c.txt 25 +7 7 c.txt 30 +8 8 c.txt 35 +9 9 c.txt 40 +10 10 c.txt 45 +11 11 c.txt 52 +12 12 c.txt 59 +13 13 c.txt 66 +14 14 c.txt 73 +15 15 c.txt 80 +16 16 c.txt 87 +17 17 c.txt 94 +18 18 c.txt 101 +19 19 c.txt 108 +20 20 c.txt 115 +100 100 c.txt 122 +101 101 c.txt 131 +102 102 c.txt 140 +103 103 c.txt 149 +104 104 c.txt 158 +105 105 c.txt 167 +106 106 c.txt 176 +107 107 c.txt 185 +108 108 c.txt 194 +109 109 c.txt 203 +110 110 c.txt 212 +111 111 c.txt 221 +112 112 c.txt 230 +113 113 c.txt 239 +114 114 c.txt 248 +115 115 c.txt 257 +116 116 c.txt 266 +117 117 c.txt 275 +118 118 c.txt 284 +119 119 c.txt 293 +120 120 c.txt 302 +1 1 e.txt 0 +2 2 e.txt 5 +3 3 e.txt 10 +4 4 e.txt 15 +5 5 e.txt 20 +6 6 e.txt 25 +7 7 e.txt 30 +8 8 e.txt 35 +9 9 e.txt 40 +10 10 e.txt 45 +11 11 e.txt 52 +12 12 e.txt 59 +13 13 e.txt 66 +14 14 e.txt 73 +15 15 e.txt 80 +16 16 e.txt 87 +17 17 e.txt 94 +18 18 e.txt 101 +19 19 e.txt 108 +20 20 e.txt 115 +100 100 e.txt 122 +101 101 e.txt 131 +102 102 e.txt 140 +103 103 e.txt 149 +104 104 e.txt 158 +105 105 e.txt 167 +106 106 e.txt 176 +107 107 e.txt 185 +108 108 e.txt 194 +109 109 e.txt 203 +110 110 e.txt 212 +111 111 e.txt 221 +112 112 e.txt 230 +113 113 e.txt 239 +114 114 e.txt 248 +115 115 e.txt 257 +116 116 e.txt 266 +117 117 e.txt 275 +118 118 e.txt 284 +119 119 e.txt 293 +120 120 e.txt 302 +OK diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh new file mode 100755 index 00000000000..478b320bfd9 --- /dev/null +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -0,0 +1,62 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +mkdir -p ${user_files_path}/logs/ + +rm -rf ${user_files_path}/logs/* + +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt + +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + +# touch does not change file content, no event +touch ${user_files_path}/logs/a.txt + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt + +rm ${user_files_path}/logs/d.txt + +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +${CLICKHOUSE_CLIENT} --query "detach table t;" +${CLICKHOUSE_CLIENT} --query "attach table t;" + +# should no records return +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +truncate ${user_files_path}/logs/a.txt --size 0 + +# exception happend +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' + +${CLICKHOUSE_CLIENT} --query "drop table file_log;" + +rm -rf ${user_files_path}/logs From 30527879b9626e1c7591b990bbab4f7226d647c9 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 12:19:19 +0000 Subject: [PATCH 172/950] fix style --- src/Storages/FileLog/StorageFileLog.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 9b64be4c99b..ebc8bb9ba8a 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -34,7 +34,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int CANNOT_GET_FILE_STAT; @@ -375,7 +374,7 @@ void StorageFileLog::openFilesAndSetPos() { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); } - /// updte file end at the monment, used in ReadBuffer and serialize + /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; context.reader.seekg(meta.last_writen_position); @@ -557,7 +556,7 @@ bool StorageFileLog::streamToViews() { block_io.out->write(block); rows += block.rows(); - /// During files open, also save file end at the openning moment + /// During files open, also save file end at the opening moment serialize(true); } block_io.out->writeSuffix(); From a1e61ebedf51a2ec30e50a9342007c0ef6be1a41 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 12:21:20 +0000 Subject: [PATCH 173/950] udpate tests --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 63450a83311..3500aabec65 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index eae9821d950..f9ecf1e7101 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 478b320bfd9..56c5dffe365 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu From f8f22f9a513f0a78b1f3798dc82dbf1eaf6377e1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 13:07:31 +0000 Subject: [PATCH 174/950] fix test --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02024_storage_filelog_mv.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 3500aabec65..fa36775a731 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') for i in {1..20} do diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index f9ecf1e7101..7e843075181 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p ${user_files_path}/logs/ diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index d6d912624a8..f73dc988c92 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 56c5dffe365..8e9769d060d 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p ${user_files_path}/logs/ From 6c4b0f2d37d0ced4e2302b93edd858fed6a37e80 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 13:37:57 +0000 Subject: [PATCH 175/950] drop table immediately fix style update --- src/Databases/DatabaseAtomic.cpp | 5 +++++ src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Storages/FileLog/FileLogSource.cpp | 7 +------ src/Storages/FileLog/StorageFileLog.cpp | 3 ++- src/Storages/FileLog/StorageFileLog.h | 2 ++ src/Storages/IStorage.h | 2 ++ .../0_stateless/02025_storage_filelog_virtual_col.sh | 4 ++-- 7 files changed, 15 insertions(+), 10 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 5c75f6f1036..26f694cb5dd 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -140,6 +140,11 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na if (table->storesDataOnDisk()) tryRemoveSymlink(table_name); + if (table->dropTableImmediately()) + { + table->drop(); + } + /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f273f8a165d..c87a9b91640 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -898,7 +898,7 @@ void DatabaseCatalog::dropTableDataTask() void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) { - if (table.table) + if (table.table && !table.table->dropTableImmediately()) { table.table->drop(); } diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index c272df1c2b2..a03ad1a4c51 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -11,12 +11,7 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -const auto MAX_FAILED_POLL_ATTEMPTS = 10; +static constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; FileLogSource::FileLogSource( StorageFileLog & storage_, diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ebc8bb9ba8a..ca7df3c5e4b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -39,7 +39,8 @@ namespace ErrorCodes extern const int CANNOT_GET_FILE_STAT; extern const int NOT_REGULAR_FILE; extern const int READ_META_FILE_FAILED; - extern const int FILE_STREAM_ERROR; + extern const int FILE_STREAM_ERROR; + extern const int LOGICAL_ERROR; } namespace diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index d715fbbfefc..4daf2bd175a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -44,6 +44,8 @@ public: void drop() override; + bool dropTableImmediately() const override { return true; } + const auto & getFormatName() const { return format_name; } enum class FileStatus diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 90cb963e064..baf57c40185 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -578,6 +578,8 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } + virtual bool dropTableImmediately() const { return false; } + private: /// Lock required for alter queries (lockForAlter). Always taken for write /// (actually can be replaced with std::mutex, but for consistency we use diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 8e9769d060d..c5e1553c41a 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -46,8 +46,8 @@ rm ${user_files_path}/logs/d.txt ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" -${CLICKHOUSE_CLIENT} --query "detach table t;" -${CLICKHOUSE_CLIENT} --query "attach table t;" +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" From 69985ac4ae928ebe762f47dcbebd31c63ea7b7c5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 02:27:24 +0000 Subject: [PATCH 176/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ca7df3c5e4b..b9e9566cddc 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -349,23 +349,23 @@ void StorageFileLog::openFilesAndSetPos() { for (const auto & file : file_infos.file_names) { - auto & context = file_infos.context_by_name.at(file); - if (context.status != FileStatus::NO_CHANGE) + auto & file_ctx = file_infos.context_by_name.at(file); + if (file_ctx.status != FileStatus::NO_CHANGE) { - context.reader = std::ifstream(getFullDataPath(file)); - if (!context.reader.good()) + file_ctx.reader = std::ifstream(getFullDataPath(file)); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); } - context.reader.seekg(0, context.reader.end); - if (!context.reader.good()) + file_ctx.reader.seekg(0, file_ctx.reader.end); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); } - auto file_end = context.reader.tellg(); - if (!context.reader.good()) + auto file_end = file_ctx.reader.tellg(); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file); } @@ -378,8 +378,8 @@ void StorageFileLog::openFilesAndSetPos() /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; - context.reader.seekg(meta.last_writen_position); - if (!context.reader.good()) + file_ctx.reader.seekg(meta.last_writen_position); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); } From c8062ad3971de46866bac10485a0de5194199aaa Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 04:25:59 +0000 Subject: [PATCH 177/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index b9e9566cddc..5507180807b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -294,8 +294,8 @@ Pipe StorageFileLog::read( auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); + /// When call closeFilesAndStoreMeta() ? openFilesAndSetPos(); - serialize(true); Pipes pipes; pipes.reserve(max_streams_number); @@ -352,7 +352,7 @@ void StorageFileLog::openFilesAndSetPos() auto & file_ctx = file_infos.context_by_name.at(file); if (file_ctx.status != FileStatus::NO_CHANGE) { - file_ctx.reader = std::ifstream(getFullDataPath(file)); + file_ctx.reader.open(getFullDataPath(file)); if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); @@ -385,6 +385,7 @@ void StorageFileLog::openFilesAndSetPos() } } } + serialize(true); } void StorageFileLog::closeFilesAndStoreMeta() @@ -522,7 +523,6 @@ bool StorageFileLog::streamToViews() auto block_io = interpreter.execute(); openFilesAndSetPos(); - serialize(); Pipes pipes; pipes.reserve(max_streams_number); @@ -562,7 +562,7 @@ bool StorageFileLog::streamToViews() } block_io.out->writeSuffix(); - serialize(); + closeFilesAndStoreMeta(); UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", From 1eb96c2fceef8b1c9f904c55bfb55993e79f9bfd Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 04:39:50 +0000 Subject: [PATCH 178/950] fix --- src/Storages/FileLog/FileLogSource.cpp | 16 ++++++++++++++++ src/Storages/FileLog/FileLogSource.h | 5 +++++ src/Storages/FileLog/StorageFileLog.cpp | 7 ------- src/Storages/FileLog/StorageFileLog.h | 6 +++--- 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index a03ad1a4c51..590630658ca 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -27,11 +27,27 @@ FileLogSource::FileLogSource( , context(context_) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) + , stream_number(stream_number_) + , max_streams_number(max_streams_number_) , non_virtual_header(metadata_snapshot_->getSampleBlockNonMaterialized()) , virtual_header( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); + /// The last FileLogSource responsible for open files + if (stream_number == max_streams_number - 1) + { + storage.openFilesAndSetPos(); + } +} + +FileLogSource::~FileLogSource() +{ + /// The last FileLogSource responsible for close files + if (stream_number == max_streams_number - 1) + { + storage.closeFilesAndStoreMeta(); + } } Chunk FileLogSource::generate() diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 7f7d75bfaad..115c68f2e25 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -28,6 +28,8 @@ public: bool noRecords() { return !buffer || buffer->noRecords(); } + virtual ~FileLogSource() override; + protected: Chunk generate() override; @@ -39,6 +41,9 @@ private: size_t poll_time_out; + size_t stream_number; + size_t max_streams_number; + std::unique_ptr buffer; Block non_virtual_header; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 5507180807b..5eec284b92b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -294,9 +294,6 @@ Pipe StorageFileLog::read( auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); - /// When call closeFilesAndStoreMeta() ? - openFilesAndSetPos(); - Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) @@ -522,8 +519,6 @@ bool StorageFileLog::streamToViews() InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); - openFilesAndSetPos(); - Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) @@ -562,8 +557,6 @@ bool StorageFileLog::streamToViews() } block_io.out->writeSuffix(); - closeFilesAndStoreMeta(); - UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 4daf2bd175a..58e4dc9d6eb 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -93,6 +93,9 @@ public: static UInt64 getInode(const String & file_name); + void openFilesAndSetPos(); + void closeFilesAndStoreMeta(); + protected: StorageFileLog( const StorageID & table_id_, @@ -154,9 +157,6 @@ private: bool updateFileInfos(); - void openFilesAndSetPos(); - void closeFilesAndStoreMeta(); - /// Serialize all file meta void serialize(bool with_end_pos = false) const; From 56be3fab0035eb46cab41f376d896c45e43b41e7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 07:17:16 +0000 Subject: [PATCH 179/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 5eec284b92b..2acff24059c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -75,9 +75,9 @@ StorageFileLog::StorageFileLog( loadMetaFiles(attach); loadFiles(); - assert( - file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() - == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); if (path_is_directory) directory_watch = std::make_unique(path); @@ -738,9 +738,9 @@ bool StorageFileLog::updateFileInfos() file_infos.file_names.swap(valid_files); /// These file infos should always have same size(one for one) - assert( - file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() - == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); return events.empty() || file_infos.file_names.empty(); } From 5c2a724a4d616f5ee17884dd9bfd9bfd312aabf4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Sep 2021 11:18:01 +0300 Subject: [PATCH 180/950] Add init worker script --- tests/ci/init_worker.sh | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 tests/ci/init_worker.sh diff --git a/tests/ci/init_worker.sh b/tests/ci/init_worker.sh new file mode 100644 index 00000000000..44cfc89f758 --- /dev/null +++ b/tests/ci/init_worker.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash +set -euo pipefail + +export DEBIAN_FRONTEND=noninteractive +export RUNNER_VERSION=2.283.1 +export RUNNER_HOME=/home/ubuntu/actions-runner + +apt-get update + +apt-get install --yes --no-install-recommends \ + apt-transport-https \ + ca-certificates \ + curl \ + gnupg \ + lsb-release \ + python3-pip + +curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg + +echo "deb [arch=amd64 signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null + +apt-get update + +apt-get install --yes --no-install-recommends docker-ce docker-ce-cli containerd.io + +usermod -aG docker ubuntu + +pip install boto3 pygithub requests urllib3 unidiff + +mkdir -p $RUNNER_HOME && cd $RUNNER_HOME + +curl -O -L https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/actions-runner-linux-x64-$RUNNER_VERSION.tar.gz + +tar xzf ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz +rm -f ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz + +./bin/installdependencies.sh From 30428ad1b068ed170c2cf6d14f3a981bf582977f Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 08:44:48 +0000 Subject: [PATCH 181/950] fix --- src/Storages/FileLog/ReadBufferFromFileLog.h | 4 ++-- src/Storages/FileLog/StorageFileLog.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 47a2770550d..8a3fdf235d7 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -78,8 +78,8 @@ private: Records records; Records::const_iterator current; - String current_file; - UInt64 current_offset; + String current_file = {}; + UInt64 current_offset = 0; using TaskThread = BackgroundSchedulePool::TaskHolder; diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 58e4dc9d6eb..ec81558438c 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -66,7 +66,7 @@ public: { String file_name; UInt64 last_writen_position = 0; - UInt64 last_open_end; + UInt64 last_open_end = 0; }; using FileNameToInode = std::unordered_map; From 814f6d5068af5504366e128686900c7a8864e244 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 12:09:01 +0000 Subject: [PATCH 182/950] fix --- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index bfcfb3a2385..e39fd84acfe 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -23,7 +23,7 @@ ReadBufferFromFileLog::ReadBufferFromFileLog( size_t stream_number_, size_t max_streams_number_) : ReadBuffer(nullptr, 0) - , log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number))) + , log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number_))) , storage(storage_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) From b3325772f78e5b91ca6aaaee9cc1dd9beafda089 Mon Sep 17 00:00:00 2001 From: Viachaslau Boben Date: Mon, 6 Sep 2021 02:25:22 +0300 Subject: [PATCH 183/950] Add normalizeUTF8 function with NFC normalization --- src/Common/ErrorCodes.cpp | 1 + src/Functions/normalizeString.cpp | 126 ++++++++++++++++++ src/Functions/registerFunctionsString.cpp | 8 ++ .../02011_normalize_utf8.reference | 3 + .../0_stateless/02011_normalize_utf8.sql | 19 +++ 5 files changed, 157 insertions(+) create mode 100644 src/Functions/normalizeString.cpp create mode 100644 tests/queries/0_stateless/02011_normalize_utf8.reference create mode 100644 tests/queries/0_stateless/02011_normalize_utf8.sql diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 53276f5b196..b6d9b65c28b 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -588,6 +588,7 @@ M(618, LZ4_DECODER_FAILED) \ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ + M(621, CANNOT_NORMALIZE_STRING) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Functions/normalizeString.cpp b/src/Functions/normalizeString.cpp new file mode 100644 index 00000000000..178c2dc2cf1 --- /dev/null +++ b/src/Functions/normalizeString.cpp @@ -0,0 +1,126 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "common/logger_useful.h" +#include "Columns/ColumnString.h" +#include "Parsers/IAST_fwd.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_NORMALIZE_STRING; +} + +namespace +{ + +struct NormalizeUTF8Impl +{ + + static void vector(const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + UErrorCode err = U_ZERO_ERROR; + + const UNormalizer2 *normalizer = unorm2_getNFCInstance(&err); + if (U_FAILURE(err)) { + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); + } + + size_t size = offsets.size(); + res_offsets.resize(size); + + ColumnString::Offset current_from_offset = 0; + ColumnString::Offset current_to_offset = 0; + + icu::UnicodeString to_string; + + PODArray from_uchars; + PODArray to_uchars; + + for (size_t i = 0; i < size; ++i) + { + size_t from_size = offsets[i] - current_from_offset - 1; + + from_uchars.resize(from_size + 1); + int32_t from_code_points; + u_strFromUTF8( + from_uchars.data(), + from_uchars.size(), + &from_code_points, + reinterpret_cast(&data[current_from_offset]), + from_size, + &err); + if (U_FAILURE(err)) { + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); + } + + // NFC should produce no more than 3x code points + // https://unicode.org/faq/normalization.html#12 + to_uchars.resize(from_code_points * 3 + 1); + + int32_t to_code_points = unorm2_normalize( + normalizer, + from_uchars.data(), + from_code_points, + to_uchars.data(), + to_uchars.size(), + &err); + if (U_FAILURE(err)) { + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); + } + + size_t max_to_size = current_to_offset + 2 * to_code_points + 1; + if (res_data.size() < max_to_size) { + res_data.resize(max_to_size); + } + + int32_t to_size; + u_strToUTF8( + reinterpret_cast(&res_data[current_to_offset]), + res_data.size() - current_to_offset, + &to_size, + to_uchars.data(), + to_code_points, + &err); + if (U_FAILURE(err)) { + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); + } + + current_to_offset += to_size; + res_data[current_to_offset] = 0; + ++current_to_offset; + res_offsets[i] = current_to_offset; + + current_from_offset = offsets[i]; + } + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception("Cannot apply function normalizeUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN); + } +}; + +struct NameNormalizeUTF8 +{ + static constexpr auto name = "normalizeUTF8"; +}; + +using FunctionNormalizeUTF8 = FunctionStringToString; +} + +void registerFunctionNormalizeUTF8(FunctionFactory & factory) { + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index ba6a294abba..f2439a3373b 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -52,6 +52,10 @@ void registerFunctionSynonyms(FunctionFactory &); void registerFunctionLemmatize(FunctionFactory &); #endif +#if USE_ICU +void registerFunctionNormalizeUTF8(FunctionFactory &); +#endif + void registerFunctionsString(FunctionFactory & factory) { registerFunctionRepeat(factory); @@ -97,6 +101,10 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionSynonyms(factory); registerFunctionLemmatize(factory); #endif + +#if USE_ICU + registerFunctionNormalizeUTF8(factory); +#endif } } diff --git a/tests/queries/0_stateless/02011_normalize_utf8.reference b/tests/queries/0_stateless/02011_normalize_utf8.reference new file mode 100644 index 00000000000..6878a38ca0d --- /dev/null +++ b/tests/queries/0_stateless/02011_normalize_utf8.reference @@ -0,0 +1,3 @@ +ё ё 2 4 ё ё 2 2 +ё 4 ё 2 +ё 2 ё 2 diff --git a/tests/queries/0_stateless/02011_normalize_utf8.sql b/tests/queries/0_stateless/02011_normalize_utf8.sql new file mode 100644 index 00000000000..c28a0c0a794 --- /dev/null +++ b/tests/queries/0_stateless/02011_normalize_utf8.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS normalize_test; +CREATE TABLE normalize_test (value String) ENGINE = MergeTree ORDER BY value; + +SELECT + 'ё' AS norm, + 'ё' AS denorm, + length(norm), + length(denorm), + normalizeUTF8(norm), + normalizeUTF8(denorm), + length(normalizeUTF8(norm)), + length(normalizeUTF8(denorm)); + +INSERT INTO normalize_test (value) VALUES ('ё'); +INSERT INTO normalize_test (value) VALUES ('ё'); + +SELECT value, length(value), normalizeUTF8(value) AS normalized, length(normalized) FROM normalize_test; + +SELECT char(228) AS value, normalizeUTF8(value); -- { serverError 619 } From df96b3e65695285760adfadaf4bf60fd8d812324 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 28 Sep 2021 02:06:16 +0000 Subject: [PATCH 184/950] fix test --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index fa36775a731..2cf23957c9e 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: long, no-parallel set -eu diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 7e843075181..095c0f05c40 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: long, no-parallel set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index c5e1553c41a..076e2af44c2 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: long, no-parallel set -eu From 1f90c8dc220dd1cfe869f8f6175458a31bbe35a5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 28 Sep 2021 05:20:02 +0000 Subject: [PATCH 185/950] refactor fix style --- .../FileLog/FileLogDirectoryWatcher.cpp | 10 +++---- src/Storages/FileLog/StorageFileLog.cpp | 27 ++++++------------- 2 files changed, 13 insertions(+), 24 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index b27b4719933..5bec4fff650 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -43,7 +43,7 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director de.callback = "onItemAdded"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } @@ -54,7 +54,7 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct de.callback = "onItemRemoved"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } @@ -65,7 +65,7 @@ void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::Direc de.callback = "onItemModified"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) @@ -75,7 +75,7 @@ void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::Dire de.callback = "onItemMovedFrom"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) @@ -85,7 +85,7 @@ void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::Direct de.callback = "onItemMovedTo"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 2acff24059c..d5d86eec0b8 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -528,7 +528,7 @@ bool StorageFileLog::streamToViews() auto convert_actions_dag = ActionsDAG::makeConvertingActions( pipe.getHeader().getColumnsWithTypeAndName(), - block_io.out->getHeader().getColumnsWithTypeAndName(), + block_io.pipeline.getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto actions = std::make_shared( @@ -538,28 +538,17 @@ bool StorageFileLog::streamToViews() pipes.emplace_back(std::move(pipe)); } - QueryPipeline pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); + auto input= Pipe::unitePipes(std::move(pipes)); - assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); + assertBlocksHaveEqualStructure(input.getHeader(), block_io.pipeline.getHeader(), "StorageFileLog streamToViews"); - size_t rows = 0; + block_io.pipeline.complete(std::move(input)); - PullingPipelineExecutor executor(pipeline); - Block block; - block_io.out->writePrefix(); - while (executor.pull(block)) - { - block_io.out->write(block); - rows += block.rows(); - /// During files open, also save file end at the opening moment - serialize(true); - } - block_io.out->writeSuffix(); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); UInt64 milliseconds = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", - formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); + LOG_DEBUG(log, "Pushing data to {} took {} ms.", table_id.getNameForLogs(), milliseconds); return updateFileInfos(); } From 762904adbda95dc24b771250b1f32ccd404db739 Mon Sep 17 00:00:00 2001 From: Viachaslau Boben Date: Mon, 27 Sep 2021 18:45:04 +0300 Subject: [PATCH 186/950] Add nfd and perf test --- src/Functions/normalizeString.cpp | 114 +++++++++++++----- tests/performance/normalize_utf8.xml | 15 +++ .../02011_normalize_utf8.reference | 12 +- .../0_stateless/02011_normalize_utf8.sql | 51 ++++++-- 4 files changed, 148 insertions(+), 44 deletions(-) create mode 100644 tests/performance/normalize_utf8.xml diff --git a/src/Functions/normalizeString.cpp b/src/Functions/normalizeString.cpp index 178c2dc2cf1..5beca566cd1 100644 --- a/src/Functions/normalizeString.cpp +++ b/src/Functions/normalizeString.cpp @@ -1,6 +1,10 @@ +#if !defined(ARCADIA_BUILD) +# include "config_core.h" +#endif + +#if USE_ICU #include #include -#include #include #include #include @@ -15,12 +19,67 @@ namespace DB namespace ErrorCodes { + extern const int ILLEGAL_COLUMN; extern const int CANNOT_NORMALIZE_STRING; } namespace { +// Expansion factors are specified for UTF-32, since icu uses UTF-32 for normalization +// Maximum expansion factors for different normalization forms +// https://unicode.org/faq/normalization.html#12 + +struct NormalizeNFCImpl +{ + static constexpr auto name = "normalizeUTF8NFC"; + + static constexpr auto expansionFactor = 3; + + static const UNormalizer2 *getNormalizer(UErrorCode *err) + { + return unorm2_getNFCInstance(err); + } +}; + +struct NormalizeNFDImpl +{ + static constexpr auto name = "normalizeUTF8NFD"; + + static constexpr auto expansionFactor = 4; + + static const UNormalizer2 *getNormalizer(UErrorCode *err) + { + return unorm2_getNFDInstance(err); + } +}; + +struct NormalizeNFKCImpl +{ + static constexpr auto name = "normalizeUTF8NFKC"; + + static constexpr auto expansionFactor = 18; + + static const UNormalizer2 *getNormalizer(UErrorCode *err) + { + return unorm2_getNFKCInstance(err); + } +}; + + +struct NormalizeNFKDImpl +{ + static constexpr auto name = "normalizeUTF8NFKD"; + + static constexpr auto expansionFactor = 18; + + static const UNormalizer2 *getNormalizer(UErrorCode *err) + { + return unorm2_getNFKDInstance(err); + } +}; + +template struct NormalizeUTF8Impl { @@ -31,10 +90,9 @@ struct NormalizeUTF8Impl { UErrorCode err = U_ZERO_ERROR; - const UNormalizer2 *normalizer = unorm2_getNFCInstance(&err); - if (U_FAILURE(err)) { - throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); - } + const UNormalizer2 *normalizer = NormalizeImpl::getNormalizer(&err); + if (U_FAILURE(err)) + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (getNormalizer): {}", u_errorName(err)); size_t size = offsets.size(); res_offsets.resize(size); @@ -60,13 +118,10 @@ struct NormalizeUTF8Impl reinterpret_cast(&data[current_from_offset]), from_size, &err); - if (U_FAILURE(err)) { - throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); - } + if (U_FAILURE(err)) + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (strFromUTF8): {}", u_errorName(err)); - // NFC should produce no more than 3x code points - // https://unicode.org/faq/normalization.html#12 - to_uchars.resize(from_code_points * 3 + 1); + to_uchars.resize(from_code_points * NormalizeImpl::expansionFactor + 1); int32_t to_code_points = unorm2_normalize( normalizer, @@ -75,14 +130,12 @@ struct NormalizeUTF8Impl to_uchars.data(), to_uchars.size(), &err); - if (U_FAILURE(err)) { - throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); - } + if (U_FAILURE(err)) + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (normalize): {}", u_errorName(err)); - size_t max_to_size = current_to_offset + 2 * to_code_points + 1; - if (res_data.size() < max_to_size) { + size_t max_to_size = current_to_offset + 4 * to_code_points + 1; + if (res_data.size() < max_to_size) res_data.resize(max_to_size); - } int32_t to_size; u_strToUTF8( @@ -92,9 +145,8 @@ struct NormalizeUTF8Impl to_uchars.data(), to_code_points, &err); - if (U_FAILURE(err)) { - throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err)); - } + if (U_FAILURE(err)) + throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (strToUTF8): {}", u_errorName(err)); current_to_offset += to_size; res_data[current_to_offset] = 0; @@ -111,16 +163,20 @@ struct NormalizeUTF8Impl } }; -struct NameNormalizeUTF8 +using FunctionNormalizeUTF8NFC = FunctionStringToString, NormalizeNFCImpl>; +using FunctionNormalizeUTF8NFD = FunctionStringToString, NormalizeNFDImpl>; +using FunctionNormalizeUTF8NFKC = FunctionStringToString, NormalizeNFKCImpl>; +using FunctionNormalizeUTF8NFKD = FunctionStringToString, NormalizeNFKDImpl>; +} + +void registerFunctionNormalizeUTF8(FunctionFactory & factory) { - static constexpr auto name = "normalizeUTF8"; -}; - -using FunctionNormalizeUTF8 = FunctionStringToString; -} - -void registerFunctionNormalizeUTF8(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } + +#endif diff --git a/tests/performance/normalize_utf8.xml b/tests/performance/normalize_utf8.xml new file mode 100644 index 00000000000..de9bd87fdf8 --- /dev/null +++ b/tests/performance/normalize_utf8.xml @@ -0,0 +1,15 @@ + + + hits_10m_single + + + CREATE TABLE strings (words String) ENGINE Memory + INSERT INTO strings SELECT SearchPhrase FROM hits_10m_single WHERE length(SearchPhrase) > 0 + + SELECT normalizeUTF8NFC(words) FROM strings FORMAT Null + SELECT normalizeUTF8NFD(words) FROM strings FORMAT Null + SELECT normalizeUTF8NFKC(words) FROM strings FORMAT Null + SELECT normalizeUTF8NFKD(words) FROM strings FORMAT Null + + DROP TABLE IF EXISTS strings + diff --git a/tests/queries/0_stateless/02011_normalize_utf8.reference b/tests/queries/0_stateless/02011_normalize_utf8.reference index 6878a38ca0d..b97f0ee5a01 100644 --- a/tests/queries/0_stateless/02011_normalize_utf8.reference +++ b/tests/queries/0_stateless/02011_normalize_utf8.reference @@ -1,3 +1,11 @@ ё ё 2 4 ё ё 2 2 -ё 4 ё 2 -ё 2 ё 2 +1 ё 4 ё 2 ё 4 ё 2 ё 4 +2 ё 2 ё 2 ё 4 ё 2 ё 4 +3 జ్ఞ‌ా 15 జ్ఞ‌ా 15 జ్ఞ‌ా 15 జ్ఞ‌ా 15 జ్ఞ‌ా 15 +4 本気ですか 15 本気ですか 15 本気ですか 18 本気ですか 15 本気ですか 18 +5 ﷺ 3 ﷺ 3 ﷺ 3 صلى الله عليه وسلم 33 صلى الله عليه وسلم 33 +6 ᾂ 3 ᾂ 3 ᾂ 8 ᾂ 3 ᾂ 8 +7 ΐ 2 ΐ 2 ΐ 6 ΐ 2 ΐ 6 +8 שּׁ 6 שּׁ 6 שּׁ 6 שּׁ 6 שּׁ 6 +9 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 +10 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 281 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 281 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 282 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 281 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 282 diff --git a/tests/queries/0_stateless/02011_normalize_utf8.sql b/tests/queries/0_stateless/02011_normalize_utf8.sql index c28a0c0a794..5abb6b4d8fb 100644 --- a/tests/queries/0_stateless/02011_normalize_utf8.sql +++ b/tests/queries/0_stateless/02011_normalize_utf8.sql @@ -1,19 +1,44 @@ +-- Tags: no-fasttest + DROP TABLE IF EXISTS normalize_test; -CREATE TABLE normalize_test (value String) ENGINE = MergeTree ORDER BY value; +CREATE TABLE normalize_test (id int, value String) ENGINE = MergeTree ORDER BY value; + SELECT - 'ё' AS norm, - 'ё' AS denorm, - length(norm), - length(denorm), - normalizeUTF8(norm), - normalizeUTF8(denorm), - length(normalizeUTF8(norm)), - length(normalizeUTF8(denorm)); + 'ё' AS norm, 'ё' AS denorm, + length(norm), length(denorm), + normalizeUTF8NFC(norm) AS norm_nfc, + normalizeUTF8NFC(denorm) AS denorm_nfc, + length(norm_nfc), + length(denorm_nfc); -INSERT INTO normalize_test (value) VALUES ('ё'); -INSERT INTO normalize_test (value) VALUES ('ё'); -SELECT value, length(value), normalizeUTF8(value) AS normalized, length(normalized) FROM normalize_test; +INSERT INTO normalize_test (id, value) VALUES (1, 'ё'); +INSERT INTO normalize_test (id, value) VALUES (2, 'ё'); +INSERT INTO normalize_test (id, value) VALUES (3, 'జ్ఞ‌ా'); +INSERT INTO normalize_test (id, value) VALUES (4, '本気ですか'); +INSERT INTO normalize_test (id, value) VALUES (5, 'ﷺ'); +INSERT INTO normalize_test (id, value) VALUES (6, 'ᾂ'); +INSERT INTO normalize_test (id, value) VALUES (7, 'ΐ'); +INSERT INTO normalize_test (id, value) VALUES (8, 'שּׁ'); +INSERT INTO normalize_test (id, value) VALUES (9, '𝅘𝅥𝅮'); -SELECT char(228) AS value, normalizeUTF8(value); -- { serverError 619 } + +INSERT INTO normalize_test (id, value) VALUES (10, 'Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒'); + + + +SELECT + id, value, length(value), + normalizeUTF8NFC(value) AS nfc, length(nfc) AS nfc_len, + normalizeUTF8NFD(value) AS nfd, length(nfd) AS nfd_len, + normalizeUTF8NFKC(value) AS nfkc, length(nfkc) AS nfkc_len, + normalizeUTF8NFKD(value) AS nfkd, length(nfkd) AS nfkd_len +FROM normalize_test +ORDER BY id; + + +SELECT char(228) AS value, normalizeUTF8NFC(value); -- { serverError 621 } +SELECT char(228) AS value, normalizeUTF8NFD(value); -- { serverError 621 } +SELECT char(228) AS value, normalizeUTF8NFKC(value); -- { serverError 621 } +SELECT char(228) AS value, normalizeUTF8NFKD(value); -- { serverError 621 } From 214272113f9d979331fabbad1bd8022b8a382710 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 14:52:51 +0300 Subject: [PATCH 187/950] Add lambda code for token rotation --- tests/ci/lambda/Dockerfile | 13 ++++ tests/ci/lambda/app.py | 106 +++++++++++++++++++++++++++++++ tests/ci/lambda/requirements.txt | 3 + 3 files changed, 122 insertions(+) create mode 100644 tests/ci/lambda/Dockerfile create mode 100644 tests/ci/lambda/app.py create mode 100644 tests/ci/lambda/requirements.txt diff --git a/tests/ci/lambda/Dockerfile b/tests/ci/lambda/Dockerfile new file mode 100644 index 00000000000..f53be71a893 --- /dev/null +++ b/tests/ci/lambda/Dockerfile @@ -0,0 +1,13 @@ +FROM public.ecr.aws/lambda/python:3.9 + +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + +# Install the function's dependencies using file requirements.txt +# from your project folder. + +COPY requirements.txt . +RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" + +# Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) +CMD [ "app.handler" ] diff --git a/tests/ci/lambda/app.py b/tests/ci/lambda/app.py new file mode 100644 index 00000000000..4edd3e8d08c --- /dev/null +++ b/tests/ci/lambda/app.py @@ -0,0 +1,106 @@ +#!/usr/bin/env python3 + +import requests +import argparse +import jwt +import sys +import json +import time + +def get_installation_id(jwt_token): + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get("https://api.github.com/app/installations", headers=headers) + response.raise_for_status() + data = response.json() + return data[0]['id'] + +def get_access_token(jwt_token, installation_id): + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.post(f"https://api.github.com/app/installations/{installation_id}/access_tokens", headers=headers) + response.raise_for_status() + data = response.json() + return data['token'] + +def get_runner_registration_token(access_token): + headers = { + "Authorization": f"token {access_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.post("https://api.github.com/orgs/ClickHouse/actions/runners/registration-token", headers=headers) + response.raise_for_status() + data = response.json() + return data['token'] + +def get_key_and_app_from_aws(): + import boto3 + secret_name = "clickhouse_github_secret_key_1" + session = boto3.session.Session() + client = session.client( + service_name='secretsmanager', + ) + get_secret_value_response = client.get_secret_value( + SecretId=secret_name + ) + data = json.loads(get_secret_value_response['SecretString']) + return data['clickhouse-app-key'], int(data['clickhouse-app-id']) + + +def main(github_secret_key, github_app_id, push_to_ssm, ssm_parameter_name): + payload = { + "iat": int(time.time()) - 60, + "exp": int(time.time()) + (10 * 60), + "iss": github_app_id, + } + + encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256") + installation_id = get_installation_id(encoded_jwt) + access_token = get_access_token(encoded_jwt, installation_id) + runner_registration_token = get_runner_registration_token(access_token) + + if push_to_ssm: + import boto3 + + print("Trying to put params into ssm manager") + client = boto3.client('ssm') + client.put_parameter( + Name=ssm_parameter_name, + Value=runner_registration_token, + Type='SecureString', + Overwrite=True) + else: + print("Not push token to AWS Parameter Store, just print:", runner_registration_token) + + +def handler(event, context): + private_key, app_id = get_key_and_app_from_aws() + main(private_key, app_id, True, 'github_runner_registration_token') + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description='Get new token from github to add runners') + parser.add_argument('-p', '--private-key-path', help='Path to file with private key') + parser.add_argument('-k', '--private-key', help='Private key') + parser.add_argument('-a', '--app-id', type=int, help='GitHub application ID', required=True) + parser.add_argument('--push-to-ssm', action='store_true', help='Store received token in parameter store') + parser.add_argument('--ssm-parameter-name', default='github_runner_registration_token', help='AWS paramater store parameter name') + + args = parser.parse_args() + + if not args.private_key_path and not args.private_key: + print("Either --private-key-path or --private-key must be specified", file=sys.stderr) + + if args.private_key_path and args.private_key: + print("Either --private-key-path or --private-key must be specified", file=sys.stderr) + + if args.private_key: + private_key = args.private_key + else: + with open(args.private_key_path, 'r') as key_file: + private_key = key_file.read() + + main(private_key, args.app_id, args.push_to_ssm, args.ssm_parameter_name) diff --git a/tests/ci/lambda/requirements.txt b/tests/ci/lambda/requirements.txt new file mode 100644 index 00000000000..c0dcf4a4dde --- /dev/null +++ b/tests/ci/lambda/requirements.txt @@ -0,0 +1,3 @@ +requests +PyJWT +cryptography From d70ea95b3dfd043ecdc09d2e7c1d9a5edfa16d37 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 16:28:25 +0300 Subject: [PATCH 188/950] First worker version --- tests/ci/init_worker.sh | 37 ------------------------------------- tests/ci/worker/init.sh | 18 ++++++++++++++++++ 2 files changed, 18 insertions(+), 37 deletions(-) delete mode 100644 tests/ci/init_worker.sh create mode 100644 tests/ci/worker/init.sh diff --git a/tests/ci/init_worker.sh b/tests/ci/init_worker.sh deleted file mode 100644 index 44cfc89f758..00000000000 --- a/tests/ci/init_worker.sh +++ /dev/null @@ -1,37 +0,0 @@ -#!/usr/bin/env bash -set -euo pipefail - -export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.283.1 -export RUNNER_HOME=/home/ubuntu/actions-runner - -apt-get update - -apt-get install --yes --no-install-recommends \ - apt-transport-https \ - ca-certificates \ - curl \ - gnupg \ - lsb-release \ - python3-pip - -curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg - -echo "deb [arch=amd64 signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null - -apt-get update - -apt-get install --yes --no-install-recommends docker-ce docker-ce-cli containerd.io - -usermod -aG docker ubuntu - -pip install boto3 pygithub requests urllib3 unidiff - -mkdir -p $RUNNER_HOME && cd $RUNNER_HOME - -curl -O -L https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/actions-runner-linux-x64-$RUNNER_VERSION.tar.gz - -tar xzf ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz -rm -f ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz - -./bin/installdependencies.sh diff --git a/tests/ci/worker/init.sh b/tests/ci/worker/init.sh new file mode 100644 index 00000000000..69432a0c220 --- /dev/null +++ b/tests/ci/worker/init.sh @@ -0,0 +1,18 @@ +#!/usr/bin/bash +set -euo pipefail + +echo "Running init script" +export DEBIAN_FRONTEND=noninteractive +export RUNNER_HOME=/home/ubuntu/actions-runner + +echo "Receiving token" +export RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` +export RUNNER_URL="https://github.com/ClickHouse" + +cd $RUNNER_HOME + +echo "Going to configure runner" +sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name `hostname -f` --runnergroup Default --labels 'self-hosted,Linux,X64' --work _work + +echo "Run" +sudo -u ubuntu ./run.sh From fca5775fac7e7ea878d67334734b790b0f6056b0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 18:40:06 +0300 Subject: [PATCH 189/950] Disable PVS check --- .github/workflows/main.yml | 44 +++++++++++++++++++------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 2fb0e54a8ee..05ed78d8c07 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -54,25 +54,25 @@ jobs: YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - PVS-Check: - needs: DockerHubPush - runs-on: [self-hosted] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/pvs_check - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - - name: PVS Check - env: - YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} - YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - PVS_STUDIO_KEY: ${{ secrets.PVS_STUDIO_KEY }} - TEMP_PATH: ${{runner.temp}}/pvs_check - REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse - run: mkdir -p ${{runner.temp}}/pvs_check && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 pvs_check.py +# PVS-Check: +# needs: DockerHubPush +# runs-on: [self-hosted] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/pvs_check +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# - name: PVS Check +# env: +# YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} +# YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} +# GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} +# PVS_STUDIO_KEY: ${{ secrets.PVS_STUDIO_KEY }} +# TEMP_PATH: ${{runner.temp}}/pvs_check +# REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse +# run: mkdir -p ${{runner.temp}}/pvs_check && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 pvs_check.py From 070a7cf727c5c20bb63c9ceff2532e6309053886 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 19:08:41 +0300 Subject: [PATCH 190/950] Bump From d120fdf5953f0db43138ad1b2face732c01dafe3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Sep 2021 21:42:41 +0300 Subject: [PATCH 191/950] check if query context exist before using cache --- src/Interpreters/Context.cpp | 1 + .../InterpreterSelectWithUnionQuery.cpp | 8 ++++++++ .../01162_strange_mutations.reference | 6 ++++++ .../0_stateless/01162_strange_mutations.sh | 19 +++++++++++++++++++ 4 files changed, 34 insertions(+) create mode 100644 tests/queries/0_stateless/01162_strange_mutations.reference create mode 100755 tests/queries/0_stateless/01162_strange_mutations.sh diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7e43343ab34..78f3e8440d2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2478,6 +2478,7 @@ void Context::setFormatSchemaPath(const String & path) Context::SampleBlockCache & Context::getSampleBlockCache() const { + assert(hasQueryContext()); return getQueryContext()->sample_block_cache; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 4aeaa9e4f13..e7ea08e557d 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -222,6 +222,14 @@ InterpreterSelectWithUnionQuery::~InterpreterSelectWithUnionQuery() = default; Block InterpreterSelectWithUnionQuery::getSampleBlock(const ASTPtr & query_ptr_, ContextPtr context_, bool is_subquery) { + if (!context_->hasQueryContext()) + { + if (is_subquery) + return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().subquery().analyze()).getSampleBlock(); + else + return InterpreterSelectWithUnionQuery(query_ptr_, context_, SelectQueryOptions().analyze()).getSampleBlock(); + } + auto & cache = context_->getSampleBlockCache(); /// Using query string because query_ptr changes for every internal SELECT auto key = queryToString(query_ptr_); diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference new file mode 100644 index 00000000000..64572fe3446 --- /dev/null +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -0,0 +1,6 @@ +1 +2 +0 +1 +2 +0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh new file mode 100755 index 00000000000..6e19f81c3ef --- /dev/null +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_DATABASE', '1')") + +for engine in "${engines[@]}" +do + $CLICKHOUSE_CLIENT -q "drop table if exists t" + $CLICKHOUSE_CLIENT -q "create table t (n int) engine=$engine order by n" + $CLICKHOUSE_CLIENT -q "insert into t values (1)" + $CLICKHOUSE_CLIENT -q "insert into t values (2)" + $CLICKHOUSE_CLIENT -q "select * from t order by n" + $CLICKHOUSE_CLIENT -q "alter table t delete where n global in (select * from (select * from t))" + $CLICKHOUSE_CLIENT -q "select count() from t" + $CLICKHOUSE_CLIENT -q "drop table t" +done From 95c3eb377bab413ac2d9238d7726898dedf33ee2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Sep 2021 10:47:15 +0300 Subject: [PATCH 192/950] Add finish check --- .github/workflows/main.yml | 12 ++++++++++- tests/ci/finish_check.py | 41 ++++++++++++++++++++++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) create mode 100644 tests/ci/finish_check.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 05ed78d8c07..3931bc1538d 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -37,7 +37,7 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/docker_images_check/changed_images.json - Style-Check: + StyleCheck: needs: DockerHubPush runs-on: [self-hosted] steps: @@ -76,3 +76,13 @@ jobs: # TEMP_PATH: ${{runner.temp}}/pvs_check # REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse # run: mkdir -p ${{runner.temp}}/pvs_check && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 pvs_check.py + Finish-Check: + needs: [Style-Check, DockerHubPush, CheckLabels] + runs-on: [self-hosted] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: cd $GITHUB_WORKSPACE/tests/ci && python3 finish_check.py + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py new file mode 100644 index 00000000000..b481c5b658c --- /dev/null +++ b/tests/ci/finish_check.py @@ -0,0 +1,41 @@ +#!/usr/bin/env python3 +import logging +from github import Github +import os + +NAME = 'Run Check (actions)' + +def filter_statuses(statuses): + """ + Squash statuses to latest state + 1. context="first", state="success", update_time=1 + 2. context="second", state="success", update_time=2 + 3. context="first", stat="failure", update_time=3 + =========> + 1. context="second", state="success" + 2. context="first", stat="failure" + """ + filt = {} + for status in sorted(statuses, key=lambda x: x.updated_at): + filt[status.context] = status + return filt + + +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event, need_orgs=True) + gh = Github(os.getenv("GITHUB_TOKEN")) + commit = get_commit(gh, pr_info.sha) + + url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + statuses = filter_statuses(list(commit.get_statuses())) + if NAME in statuses and statuses[NAME].state == "pending": + commit.create_status(context=NAME, description="All checks finished", state="success", target_url=url) From cb81189bf91a1d5fd7448d4df3ca66f51fb976e2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Sep 2021 10:49:25 +0300 Subject: [PATCH 193/950] Fix workflow --- .github/workflows/main.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 3931bc1538d..8cb771a0d45 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -76,8 +76,8 @@ jobs: # TEMP_PATH: ${{runner.temp}}/pvs_check # REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse # run: mkdir -p ${{runner.temp}}/pvs_check && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 pvs_check.py - Finish-Check: - needs: [Style-Check, DockerHubPush, CheckLabels] + FinishCheck: + needs: [StyleCheck, DockerHubPush, CheckLabels] runs-on: [self-hosted] steps: - name: Check out repository code From bf9ebf42112577018347975ba9d9ec023eb2bf7b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Sep 2021 10:55:12 +0300 Subject: [PATCH 194/950] Import json --- tests/ci/finish_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index b481c5b658c..1b022905cda 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 import logging from github import Github +import json import os NAME = 'Run Check (actions)' From 8d29a472fa088468584e370d0124ffb1e8f36175 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Sep 2021 11:02:36 +0300 Subject: [PATCH 195/950] Fix --- tests/ci/finish_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 1b022905cda..89139468fd6 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -1,6 +1,7 @@ #!/usr/bin/env python3 import logging from github import Github +from pr_info import PRInfo import json import os From fcebf7b9853452caaffc39d91a31d19ae55a45ba Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Wed, 29 Sep 2021 11:29:24 +0000 Subject: [PATCH 196/950] correct tests --- src/Server/HTTPHandlerFactory.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index e81955ef2b2..f6d96189d92 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -108,8 +108,7 @@ public: { addFilter([](const auto & request) { - return (request.getURI().find('?') != std::string::npos - && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + return request.getURI().find('?') != std::string::npos || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS || request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST; }); From 36b699659e466c1deaf4737f973adcfc95fe378b Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Wed, 29 Sep 2021 14:32:04 +0300 Subject: [PATCH 197/950] Update CORS.xml --- tests/config/config.d/CORS.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/config/config.d/CORS.xml b/tests/config/config.d/CORS.xml index 9dd7d402416..873821478dc 100644 --- a/tests/config/config.d/CORS.xml +++ b/tests/config/config.d/CORS.xml @@ -1,4 +1,4 @@ - +
Access-Control-Allow-Origin @@ -17,5 +17,5 @@ 86400
- + From d75136c3b1b3040b87dde90463e9a0e8a087b16b Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 29 Sep 2021 16:59:56 +0300 Subject: [PATCH 198/950] Update hash functions (SHA) en --- .../sql-reference/functions/hash-functions.md | 145 +++++++++++++++++- 1 file changed, 144 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 227e2885417..a3154e5c200 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -139,17 +139,160 @@ It works faster than intHash32. Average quality. ## SHA1 {#sha1} +Calculates SHA-1 hash from a string and returns the resulting set of bytes as [FixedString(20)](../data-types/fixedstring.md). + +**Syntax** + +``` sql +SHA1('s') +``` + +**Arguments** + +- `s` — Input string for SHA-1 hash calculation. [String](..data-types/string.md). + +**Returned value** + +- SHA-1 hash as a hex-unencoded FixedString(10). + +Type: [FixedString](../data-types/fixedstring.md). + +**Example** + +Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. + +Query: + +``` sql +SELECT hex(SHA1('abc')); +``` + +Result: + +``` text +┌─hex(SHA1('abc'))─────────────────────────┐ +│ A9993E364706816ABA3E25717850C26C9CD0D89D │ +└──────────────────────────────────────────┘ +``` + ## SHA224 {#sha224} +Calculates SHA-224 hash from a string and returns the resulting set of bytes as [FixedString(28)](../data-types/fixedstring.md). + +**Syntax** + +``` sql +SHA224('s') +``` + +**Arguments** + +- `s` — Input string for SHA-224 hash calculation. [String](..data-types/string.md). + +**Returned value** + +- SHA-224 hash as a hex-unencoded FixedString(28). + +Type: [FixedString](../data-types/fixedstring.md). + +**Example** + +Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. + +Query: + +``` sql +SELECT hex(SHA224('abc')); +``` + +Result: + +``` text +┌─hex(SHA224('abc'))───────────────────────────────────────┐ +│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │ +└──────────────────────────────────────────────────────────┘ +``` + ## SHA256 {#sha256} +Calculates SHA-256 hash from a string and returns the resulting set of bytes as [FixedString(32)](../data-types/fixedstring.md). + +**Syntax** + +``` sql +SHA256('s') +``` + +**Arguments** + +- `s` — Input string for SHA-256 hash calculation. [String](..data-types/string.md). + +**Returned value** + +- SHA-256 hash as a hex-unencoded FixedString(32). + +Type: [FixedString](../data-types/fixedstring.md). + +**Example** + +Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. + +Query: + +``` sql +SELECT hex(SHA256('abc')); +``` + +Result: + +``` text +┌─hex(SHA256('abc'))───────────────────────────────────────────────┐ +│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │ +└──────────────────────────────────────────────────────────────────┘ +``` + ## SHA512 {#sha512} -Calculates SHA-1, SHA-224, SHA-256 or SHA-512 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), FixedString(32), or FixedString(64). +Calculates SHA-512 hash from a string and returns the resulting set of bytes as [FixedString(64)](../data-types/fixedstring.md). + +**Syntax** + +``` sql +SHA512('s') +``` + The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million). We recommend using this function only in cases when you need a specific hash function and you can’t select it. Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS. +**Arguments** + +- `s` — Input string for SHA-512 hash calculation. [String](..data-types/string.md). + +**Returned value** + +- SHA-512 hash as a hex-unencoded FixedString(64). + +Type: [FixedString](../data-types/fixedstring.md). + +**Example** + +Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. + +Query: + +``` sql +SELECT hex(SHA512('abc')); +``` + +Result: + +``` text +┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## URLHash(url\[, N\]) {#urlhashurl-n} A fast, decent-quality non-cryptographic hash function for a string obtained from a URL using some type of normalization. From e312156b1c2a6a8a79177fb543c5d110ea47a058 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 29 Sep 2021 17:52:39 +0300 Subject: [PATCH 199/950] Add note about Materialized views --- docs/en/sql-reference/functions/hash-functions.md | 2 +- docs/en/sql-reference/statements/create/view.md | 7 +++---- docs/ru/sql-reference/statements/create/view.md | 5 ++++- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index a3154e5c200..dc4c749865a 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -263,7 +263,7 @@ SHA512('s') The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million). We recommend using this function only in cases when you need a specific hash function and you can’t select it. -Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS. +Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in `SELECT` queries. **Arguments** diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index b6a09e25f95..84213020925 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -50,14 +50,13 @@ When creating a materialized view with `TO [db].[table]`, you must not use `POPU A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view. !!! important "Important" - Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not `Nullable`. A safe practice would be to add aliases for every column when using Materialized views. + Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views. -!!! important "Important" Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. -If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using POPULATE, since data inserted in the table during the view creation will not be inserted in it. +If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it. -A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`. +A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`. The execution of [ALTER](../../../sql-reference/statements/alter/view.md) queries on materialized views has limitations, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view. diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index ccbf79baa73..53d75b78dd1 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -48,9 +48,12 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na Материализованное представление устроено следующим образом: при вставке данных в таблицу, указанную в SELECT-е, кусок вставляемых данных преобразуется этим запросом SELECT, и полученный результат вставляется в представление. !!! important "Важно" + + Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца. + Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление. -Если указано `POPULATE`, то при создании представления, в него будут вставлены имеющиеся данные таблицы, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Иначе, представление будет содержать только данные, вставляемые в таблицу после создания представления. Не рекомендуется использовать POPULATE, так как вставляемые в таблицу данные во время создания представления, не попадут в него. +Если указано `POPULATE`, то при создании представления, в него будут вставлены имеющиеся данные таблицы, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Иначе, представление будет содержать только данные, вставляемые в таблицу после создания представления. Не рекомендуется использовать `POPULATE`, так как вставляемые в таблицу данные во время создания представления, не попадут в него. Запрос `SELECT` может содержать `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Следует иметь ввиду, что соответствующие преобразования будут выполняться независимо, на каждый блок вставляемых данных. Например, при наличии `GROUP BY`, данные будут агрегироваться при вставке, но только в рамках одной пачки вставляемых данных. Далее, данные не будут доагрегированы. Исключение - использование ENGINE, производящего агрегацию данных самостоятельно, например, `SummingMergeTree`. From b226429435eeda0cae88b7553f471b6e413cff3d Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 29 Sep 2021 18:41:15 +0300 Subject: [PATCH 200/950] Fix links, add 512 translation. --- .../sql-reference/functions/hash-functions.md | 8 ++-- .../sql-reference/functions/hash-functions.md | 45 ++++++++++++++++++- 2 files changed, 47 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index dc4c749865a..e28594540be 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -149,7 +149,7 @@ SHA1('s') **Arguments** -- `s` — Input string for SHA-1 hash calculation. [String](..data-types/string.md). +- `s` — Input string for SHA-1 hash calculation. [String](../data-types/string.md). **Returned value** @@ -187,7 +187,7 @@ SHA224('s') **Arguments** -- `s` — Input string for SHA-224 hash calculation. [String](..data-types/string.md). +- `s` — Input string for SHA-224 hash calculation. [String](../data-types/string.md). **Returned value** @@ -225,7 +225,7 @@ SHA256('s') **Arguments** -- `s` — Input string for SHA-256 hash calculation. [String](..data-types/string.md). +- `s` — Input string for SHA-256 hash calculation. [String](../data-types/string.md). **Returned value** @@ -267,7 +267,7 @@ Even in these cases, we recommend applying the function offline and pre-calculat **Arguments** -- `s` — Input string for SHA-512 hash calculation. [String](..data-types/string.md). +- `s` — Input string for SHA-512 hash calculation. [String](../data-types/string.md). **Returned value** diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 07c741e0588..d7e86d5a540 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -143,10 +143,51 @@ SELECT groupBitXor(cityHash64(*)) FROM table ## SHA256 {#sha256} -Вычисляет SHA-1, SHA-224, SHA-256 от строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32). + + +## SHA512 {#sha512} + +Вычисляет SHA-1, SHA-224, SHA-256 хеш строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32), [FixedString(64)](../data-types/fixedstring.md) + +Вычисляет SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString(64)](../data-types/fixedstring.md) + +**Синтаксис** + +``` sql +SHA512('s') +``` + Функция работает достаточно медленно (SHA-1 - примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 - примерно 2.2 миллионов). Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать. -Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при SELECT-ах. +Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при выполнении `SELECT`. + +**Параметры** + +- `s` — входная строка для вычисления хеша SHA-512. [String](../data-types/string.md). + +**Возвращаемое значение** + +- Хеш SHA-512 в виде шестнадцатеричной некодированной строки FixedString(64). + +Тип: [FixedString](../data-types/fixedstring.md). + +**Пример** + +Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. + +Запрос: + +``` sql +SELECT hex(SHA512('abc')); +``` + +Результат: + +``` text +┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` ## URLHash(url\[, N\]) {#urlhashurl-n} From cac28833d247617804627e3059589da17c09de1d Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 29 Sep 2021 18:54:04 +0300 Subject: [PATCH 201/950] apply added config in tests --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index df62cba0ea9..936c44a4e7b 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -37,6 +37,7 @@ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/CORS.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logger.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ @@ -57,7 +58,6 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_pool_dictionary.xml $DEST_SERVER_PATH/ -ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/ From b702f7cbff638963e1d5afc1c4689f74d062d322 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 29 Sep 2021 13:37:44 +0300 Subject: [PATCH 202/950] preserve table alias when adding default database --- src/Interpreters/AddDefaultDatabaseVisitor.h | 7 ++++++- .../queries/0_stateless/01162_strange_mutations.reference | 2 ++ tests/queries/0_stateless/01162_strange_mutations.sh | 6 ++++-- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index fe3edc00957..858608acdbe 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -105,7 +105,12 @@ private: void visit(const ASTTableIdentifier & identifier, ASTPtr & ast) const { if (!identifier.compound()) - ast = std::make_shared(database_name, identifier.name()); + { + auto qualified_identifier = std::make_shared(database_name, identifier.name()); + if (!identifier.alias.empty()) + qualified_identifier->setAlias(identifier.alias); + ast = qualified_identifier; + } } void visit(ASTSubquery & subquery, ASTPtr &) const diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference index 64572fe3446..e09dccd4c4a 100644 --- a/tests/queries/0_stateless/01162_strange_mutations.reference +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -1,6 +1,8 @@ 1 2 +2 0 1 2 +2 0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index 6e19f81c3ef..c4166a88e42 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_DATABASE', '1')") +declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1')") for engine in "${engines[@]}" do @@ -13,7 +13,9 @@ do $CLICKHOUSE_CLIENT -q "insert into t values (1)" $CLICKHOUSE_CLIENT -q "insert into t values (2)" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT -q "alter table t delete where n global in (select * from (select * from t))" + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select * from (select * from t where n global in (1::Int32)))" + $CLICKHOUSE_CLIENT -q "select * from t order by n" + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" $CLICKHOUSE_CLIENT -q "select count() from t" $CLICKHOUSE_CLIENT -q "drop table t" done From 02205492e5cef5119455bbed48d339349cb4575e Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 29 Sep 2021 19:55:11 +0300 Subject: [PATCH 203/950] Update hash-functions.md Add ru translation. --- .../sql-reference/functions/hash-functions.md | 110 +++++++++++++++++- 1 file changed, 107 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index d7e86d5a540..98b5ed6df27 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -139,16 +139,120 @@ SELECT groupBitXor(cityHash64(*)) FROM table ## SHA1 {#sha1} +Вычисляет SHA-1 хеш строки и возвращает полученный набор байт в виде [FixedString(20)](../data-types/fixedstring.md). + +**Синтаксис** + +``` sql +SHA1('s') +``` + +**Параметры** + +- `s` — входная строка для вычисления хеша SHA-1. [String](../data-types/string.md). + +**Возвращаемое значение** + +- Хеш SHA-1 в виде шестнадцатеричной некодированной строки FixedString(20). + +Тип: [FixedString](../data-types/fixedstring.md). + +**Пример** + +Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. + +Запрос: + +``` sql +SELECT hex(SHA1('abc')); +``` + +Результат: + +``` text +┌─hex(SHA1('abc'))─────────────────────────┐ +│ A9993E364706816ABA3E25717850C26C9CD0D89D │ +└──────────────────────────────────────────┘ +``` + ## SHA224 {#sha224} +Вычисляет SHA-224 хеш строки и возвращает полученный набор байт в виде [FixedString(28)](../data-types/fixedstring.md). + +**Синтаксис** + +``` sql +SHA224('s') +``` + +**Параметры** + +- `s` — входная строка для вычисления хеша SHA-224. [String](../data-types/string.md). + +**Возвращаемое значение** + +- Хеш SHA-224 в виде шестнадцатеричной некодированной строки FixedString(28). + +Тип: [FixedString](../data-types/fixedstring.md). + +**Пример** + +Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. + +Запрос: + +``` sql +SELECT hex(SHA224('abc')); +``` + +Результат: + +``` text +┌─hex(SHA224('abc'))───────────────────────────────────────┐ +│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │ +└──────────────────────────────────────────────────────────┘ +``` + ## SHA256 {#sha256} +Вычисляет SHA-256 хеш строки и возвращает полученный набор байт в виде [FixedString(32)](../data-types/fixedstring.md). +**Синтаксис** + +``` sql +SHA256('s') +``` + +**Параметры** + +- `s` — входная строка для вычисления хеша SHA-256. [String](../data-types/string.md). + +**Возвращаемое значение** + +- Хеш SHA-256 в виде шестнадцатеричной некодированной строки FixedString(32). + +Тип: [FixedString](../data-types/fixedstring.md). + +**Пример** + +Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. + +Запрос: + +``` sql +SELECT hex(SHA256('abc')); +``` + +Результат: + +``` text +┌─hex(SHA256('abc'))───────────────────────────────────────────────┐ +│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │ +└──────────────────────────────────────────────────────────────────┘ +``` ## SHA512 {#sha512} -Вычисляет SHA-1, SHA-224, SHA-256 хеш строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32), [FixedString(64)](../data-types/fixedstring.md) - Вычисляет SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString(64)](../data-types/fixedstring.md) **Синтаксис** @@ -157,7 +261,7 @@ SELECT groupBitXor(cityHash64(*)) FROM table SHA512('s') ``` -Функция работает достаточно медленно (SHA-1 - примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 - примерно 2.2 миллионов). +Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов). Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать. Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при выполнении `SELECT`. From 66bb857a1a7988ec0f94bc9c83668ff662b651b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 29 Sep 2021 19:11:38 +0200 Subject: [PATCH 204/950] Add test for JOIN engine deadlock --- .../02033_join_engine_deadlock.reference | 0 .../0_stateless/02033_join_engine_deadlock.sh | 71 +++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 tests/queries/0_stateless/02033_join_engine_deadlock.reference create mode 100755 tests/queries/0_stateless/02033_join_engine_deadlock.sh diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.reference b/tests/queries/0_stateless/02033_join_engine_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.sh b/tests/queries/0_stateless/02033_join_engine_deadlock.sh new file mode 100755 index 00000000000..7a4ca1c8bb1 --- /dev/null +++ b/tests/queries/0_stateless/02033_join_engine_deadlock.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +# Tags: deadlock + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +create_table () { + $CLICKHOUSE_CLIENT --query " + CREATE TABLE join_block_test + ( + id String, + num Int64 + ) + ENGINE = Join(ANY, LEFT, id) + " +} + +drop_table () { + # Force a sync drop to free the memory before ending the test + # Otherwise things get interesting if you run the test many times before the database is finally dropped + $CLICKHOUSE_CLIENT --query " + DROP TABLE join_block_test SYNC + " +} + +populate_table_bg () { + ( + $CLICKHOUSE_CLIENT --query " + INSERT INTO join_block_test + SELECT toString(number) as id, number * number as num + FROM system.numbers LIMIT 3000000 + " --lock_acquire_timeout=20 >/dev/null + ) & +} + +read_table_bg () { + ( + $CLICKHOUSE_CLIENT --query " + SELECT * + FROM + ( + SELECT toString(number) AS user_id + FROM system.numbers LIMIT 10000 OFFSET 20000 + ) AS t1 + LEFT JOIN + ( + SELECT + * + FROM join_block_test AS i1 + ANY LEFT JOIN + ( + SELECT * + FROM join_block_test + ) AS i2 ON i1.id = toString(i2.num) + ) AS t2 ON t1.user_id = t2.id + " --lock_acquire_timeout=20 >/dev/null + ) & +} + +create_table +for _ in {1..5}; +do + populate_table_bg + sleep 0.05 + read_table_bg + sleep 0.05 +done + +wait +drop_table From 0ee5c0bff570ec676a394e2e60dc934b1e640b53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 29 Sep 2021 19:30:07 +0200 Subject: [PATCH 205/950] Use RWLock in StorageJoin to avoid deadlocks --- src/Functions/FunctionJoinGet.cpp | 6 +-- src/Functions/FunctionJoinGet.h | 16 +++++--- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/HashJoin.h | 7 ++-- src/Storages/IStorage.h | 1 + src/Storages/StorageJoin.cpp | 53 +++++++++++++++---------- src/Storages/StorageJoin.h | 13 +++--- src/Storages/StorageSet.cpp | 21 +++++----- src/Storages/StorageSet.h | 8 ++-- 10 files changed, 76 insertions(+), 53 deletions(-) diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index f0dff0ac7e4..df131538275 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -25,14 +25,14 @@ ColumnPtr ExecutableFunctionJoinGet::executeImpl(const ColumnsWithTypeA auto key = arguments[i]; keys.emplace_back(std::move(key)); } - return storage_join->joinGet(keys, result_columns).column; + return storage_join->joinGet(keys, result_columns, getContext()).column; } template ExecutableFunctionPtr FunctionJoinGet::prepare(const ColumnsWithTypeAndName &) const { Block result_columns {{return_type->createColumn(), return_type, attr_name}}; - return std::make_unique>(table_lock, storage_join, result_columns); + return std::make_unique>(getContext(), table_lock, storage_join, result_columns); } static std::pair, String> @@ -89,7 +89,7 @@ FunctionBasePtr JoinGetOverloadResolver::buildImpl(const ColumnsWithTyp auto return_type = storage_join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); - return std::make_unique>(table_lock, storage_join, attr_name, argument_types, return_type); + return std::make_unique>(getContext(), table_lock, storage_join, attr_name, argument_types, return_type); } void registerFunctionJoinGet(FunctionFactory & factory) diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index 3ddab51e2d9..2dd0cb9fdea 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -14,13 +14,15 @@ class StorageJoin; using StorageJoinPtr = std::shared_ptr; template -class ExecutableFunctionJoinGet final : public IExecutableFunction +class ExecutableFunctionJoinGet final : public IExecutableFunction, WithContext { public: - ExecutableFunctionJoinGet(TableLockHolder table_lock_, + ExecutableFunctionJoinGet(ContextPtr context_, + TableLockHolder table_lock_, StorageJoinPtr storage_join_, const DB::Block & result_columns_) - : table_lock(std::move(table_lock_)) + : WithContext(context_) + , table_lock(std::move(table_lock_)) , storage_join(std::move(storage_join_)) , result_columns(result_columns_) {} @@ -42,15 +44,17 @@ private: }; template -class FunctionJoinGet final : public IFunctionBase +class FunctionJoinGet final : public IFunctionBase, WithContext { public: static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; - FunctionJoinGet(TableLockHolder table_lock_, + FunctionJoinGet(ContextPtr context_, + TableLockHolder table_lock_, StorageJoinPtr storage_join_, String attr_name_, DataTypes argument_types_, DataTypePtr return_type_) - : table_lock(std::move(table_lock_)) + : WithContext(context_) + , table_lock(std::move(table_lock_)) , storage_join(storage_join_) , attr_name(std::move(attr_name_)) , argument_types(std::move(argument_types_)) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 566ee60a3e6..89d7624f203 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -938,7 +938,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin( if (auto storage = analyzed_join->getStorageJoin()) { std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, {}); - return storage->getJoinLocked(analyzed_join); + return storage->getJoinLocked(analyzed_join, getContext()); } joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 07872df8ce5..d88df9d3e30 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -744,7 +744,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) size_t total_rows = 0; size_t total_bytes = 0; { - if (storage_join_lock.mutex()) + if (storage_join_lock) throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 07fd6d5b89f..f1f1198e7d9 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -334,9 +335,9 @@ public: /// We keep correspondence between used_flags and hash table internal buffer. /// Hash table cannot be modified during HashJoin lifetime and must be protected with lock. - void setLock(std::shared_mutex & rwlock) + void setLock(RWLockImpl::LockHolder rwlock_holder) { - storage_join_lock = std::shared_lock(rwlock); + storage_join_lock = rwlock_holder; } void reuseJoinedData(const HashJoin & join); @@ -391,7 +392,7 @@ private: /// Should be set via setLock to protect hash table from modification from StorageJoin /// If set HashJoin instance is not available for modification (addJoinedBlock) - std::shared_lock storage_join_lock; + RWLockImpl::LockHolder storage_join_lock = nullptr; void dataMapInit(MapsVariant &); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6ce17552ba1..2013cc5ecb6 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -219,6 +219,7 @@ private: /// without locks. MultiVersionStorageMetadataPtr metadata; +protected: RWLockImpl::LockHolder tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index e45183591f2..e5574708de0 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -1,13 +1,13 @@ #include #include #include +#include #include #include #include #include #include #include -#include #include #include #include @@ -67,6 +67,14 @@ StorageJoin::StorageJoin( restore(); } +RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const +{ + const String query_id = ctx ? ctx->getInitialQueryId() : RWLockImpl::NO_QUERY; + const std::chrono::milliseconds acquire_timeout + = ctx ? ctx->getSettingsRef().lock_acquire_timeout : std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC); + return tryLockTimed(lock, type, query_id, acquire_timeout); +} + SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { std::lock_guard mutate_lock(mutate_mutex); @@ -74,10 +82,10 @@ SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataP } void StorageJoin::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder&) + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx, TableExclusiveLockHolder&) { std::lock_guard mutate_lock(mutate_mutex); - std::unique_lock lock(rwlock); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx); disk->removeRecursive(path); disk->createDirectories(path); @@ -128,7 +136,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) } /// Now acquire exclusive lock and modify storage. - std::unique_lock lock(rwlock); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context); join = std::move(new_data); increment = 1; @@ -152,7 +160,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) } } -HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) const +HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const { auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) @@ -171,34 +179,36 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) analyzed_join->setRightKeys(key_names); HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns()); - join_clone->setLock(rwlock); + + RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx); + join_clone->setLock(holder); join_clone->reuseJoinedData(*join); return join_clone; } -void StorageJoin::insertBlock(const Block & block) +void StorageJoin::insertBlock(const Block & block, ContextPtr ctx) { - std::unique_lock lock(rwlock); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx); join->addJoinedBlock(block, true); } -size_t StorageJoin::getSize() const +size_t StorageJoin::getSize(ContextPtr ctx) const { - std::shared_lock lock(rwlock); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx); return join->getTotalRowCount(); } -std::optional StorageJoin::totalRows(const Settings &) const +std::optional StorageJoin::totalRows(const Settings &settings) const { - std::shared_lock lock(rwlock); + TableLockHolder holder = tryLockTimed(rwlock, RWLockImpl::Read, RWLockImpl::NO_QUERY, settings.lock_acquire_timeout); return join->getTotalRowCount(); } -std::optional StorageJoin::totalBytes(const Settings &) const +std::optional StorageJoin::totalBytes(const Settings &settings) const { - std::shared_lock lock(rwlock); + TableLockHolder holder = tryLockTimed(rwlock, RWLockImpl::Read, RWLockImpl::NO_QUERY, settings.lock_acquire_timeout); return join->getTotalByteCount(); } @@ -207,9 +217,9 @@ DataTypePtr StorageJoin::joinGetCheckAndGetReturnType(const DataTypes & data_typ return join->joinGetCheckAndGetReturnType(data_types, column_name, or_null); } -ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const +ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr ctx) const { - std::shared_lock lock(rwlock); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx); return join->joinGet(block, block_with_columns_to_add); } @@ -370,10 +380,10 @@ size_t rawSize(const StringRef & t) class JoinSource : public SourceWithProgress { public: - JoinSource(HashJoinPtr join_, std::shared_mutex & rwlock, UInt64 max_block_size_, Block sample_block_) + JoinSource(HashJoinPtr join_, TableLockHolder lock_holder_, UInt64 max_block_size_, Block sample_block_) : SourceWithProgress(sample_block_) , join(join_) - , lock(rwlock) + , lock_holder(lock_holder_) , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) { @@ -421,7 +431,7 @@ protected: private: HashJoinPtr join; - std::shared_lock lock; + TableLockHolder lock_holder; UInt64 max_block_size; Block sample_block; @@ -571,7 +581,7 @@ Pipe StorageJoin::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & /*query_info*/, - ContextPtr /*context*/, + ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned /*num_streams*/) @@ -579,7 +589,8 @@ Pipe StorageJoin::read( metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block source_sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); - return Pipe(std::make_shared(join, rwlock, max_block_size, source_sample_block)); + RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context); + return Pipe(std::make_shared(join, std::move(holder), max_block_size, source_sample_block)); } } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 6a08773ecc8..4926194433c 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -2,7 +2,9 @@ #include +#include #include +#include #include #include @@ -35,7 +37,7 @@ public: /// Return instance of HashJoin holding lock that protects from insertions to StorageJoin. /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. - HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join) const; + HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const; /// Get result type for function "joinGet(OrNull)" DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; @@ -43,7 +45,7 @@ public: /// Execute function "joinGet(OrNull)" on data block. /// Takes rwlock for read to prevent parallel StorageJoin updates during processing data block /// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin) - ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr context) const; SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; @@ -73,12 +75,13 @@ private: /// Protect state for concurrent use in insertFromBlock and joinBlock. /// Lock is stored in HashJoin instance during query and blocks concurrent insertions. - mutable std::shared_mutex rwlock; + mutable RWLock rwlock = RWLockImpl::create(); mutable std::mutex mutate_mutex; - void insertBlock(const Block & block) override; + void insertBlock(const Block & block, ContextPtr ctx) override; void finishInsert() override {} - size_t getSize() const override; + size_t getSize(ContextPtr context) const override; + RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const; protected: StorageJoin( diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index fe55123335a..c57dadf6d52 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -34,11 +34,11 @@ namespace ErrorCodes } -class SetOrJoinSink : public SinkToStorage +class SetOrJoinSink : public SinkToStorage, WithContext { public: SetOrJoinSink( - StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, + ContextPtr ctx, StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_, bool persistent_); @@ -60,6 +60,7 @@ private: SetOrJoinSink::SetOrJoinSink( + ContextPtr ctx, StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, @@ -67,6 +68,7 @@ SetOrJoinSink::SetOrJoinSink( const String & backup_file_name_, bool persistent_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) + , WithContext(ctx) , table(table_) , metadata_snapshot(metadata_snapshot_) , backup_path(backup_path_) @@ -84,7 +86,7 @@ void SetOrJoinSink::consume(Chunk chunk) /// Sort columns in the block. This is necessary, since Set and Join count on the same column order in different blocks. Block sorted_block = getHeader().cloneWithColumns(chunk.detachColumns()).sortColumns(); - table.insertBlock(sorted_block); + table.insertBlock(sorted_block, getContext()); if (persistent) backup_stream.write(sorted_block); } @@ -104,10 +106,10 @@ void SetOrJoinSink::onFinish() } -SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx) { UInt64 id = ++increment; - return std::make_shared(*this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent); + return std::make_shared(ctx, *this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent); } @@ -155,10 +157,10 @@ StorageSet::StorageSet( } -void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block.getColumnsWithTypeAndName()); } +void StorageSet::insertBlock(const Block & block, ContextPtr) { set->insertFromBlock(block.getColumnsWithTypeAndName()); } void StorageSet::finishInsert() { set->finishInsert(); } -size_t StorageSet::getSize() const { return set->getTotalRowCount(); } +size_t StorageSet::getSize(ContextPtr) const { return set->getTotalRowCount(); } std::optional StorageSet::totalRows(const Settings &) const { return set->getTotalRowCount(); } std::optional StorageSet::totalBytes(const Settings &) const { return set->getTotalByteCount(); } @@ -210,6 +212,7 @@ void StorageSetOrJoinBase::restore() void StorageSetOrJoinBase::restoreFromFile(const String & file_path) { + ContextPtr ctx = nullptr; auto backup_buf = disk->readFile(file_path); CompressedReadBuffer compressed_backup_buf(*backup_buf); NativeBlockInputStream backup_stream(compressed_backup_buf, 0); @@ -217,14 +220,14 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) backup_stream.readPrefix(); while (Block block = backup_stream.read()) - insertBlock(block); + insertBlock(block, ctx); finishInsert(); backup_stream.readSuffix(); /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. LOG_INFO(&Poco::Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", - file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize()); + file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize(ctx)); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 1166557ec8e..1b78676b6c5 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -51,10 +51,10 @@ private: void restoreFromFile(const String & file_path); /// Insert the block into the state. - virtual void insertBlock(const Block & block) = 0; + virtual void insertBlock(const Block & block, ContextPtr context) = 0; /// Call after all blocks were inserted. virtual void finishInsert() = 0; - virtual size_t getSize() const = 0; + virtual size_t getSize(ContextPtr context) const = 0; }; @@ -81,9 +81,9 @@ public: private: SetPtr set; - void insertBlock(const Block & block) override; + void insertBlock(const Block & block, ContextPtr) override; void finishInsert() override; - size_t getSize() const override; + size_t getSize(ContextPtr) const override; protected: StorageSet( From c0ba8d1a043c962c365c142e64a75a8e5db993b0 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 29 Sep 2021 21:31:18 +0300 Subject: [PATCH 206/950] Fix crosslink. --- docs/en/sql-reference/statements/create/view.md | 2 +- docs/ru/sql-reference/statements/create/view.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 84213020925..39c5760ecf3 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -50,7 +50,7 @@ When creating a materialized view with `TO [db].[table]`, you must not use `POPU A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view. !!! important "Important" - Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views. + Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views. Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index 53d75b78dd1..9eb0baf5a98 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -49,7 +49,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na !!! important "Важно" - Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца. + Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца. Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление. From 112a009b918b548d4b6d5a21caf8a857d5de46f6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 29 Sep 2021 14:40:20 +0300 Subject: [PATCH 207/950] Update worker script --- tests/ci/{lambda => token_lambda}/Dockerfile | 0 tests/ci/{lambda => token_lambda}/app.py | 0 tests/ci/{lambda => token_lambda}/requirements.txt | 0 tests/ci/worker/init.sh | 4 +++- 4 files changed, 3 insertions(+), 1 deletion(-) rename tests/ci/{lambda => token_lambda}/Dockerfile (100%) rename tests/ci/{lambda => token_lambda}/app.py (100%) rename tests/ci/{lambda => token_lambda}/requirements.txt (100%) diff --git a/tests/ci/lambda/Dockerfile b/tests/ci/token_lambda/Dockerfile similarity index 100% rename from tests/ci/lambda/Dockerfile rename to tests/ci/token_lambda/Dockerfile diff --git a/tests/ci/lambda/app.py b/tests/ci/token_lambda/app.py similarity index 100% rename from tests/ci/lambda/app.py rename to tests/ci/token_lambda/app.py diff --git a/tests/ci/lambda/requirements.txt b/tests/ci/token_lambda/requirements.txt similarity index 100% rename from tests/ci/lambda/requirements.txt rename to tests/ci/token_lambda/requirements.txt diff --git a/tests/ci/worker/init.sh b/tests/ci/worker/init.sh index 69432a0c220..2f6638f14b5 100644 --- a/tests/ci/worker/init.sh +++ b/tests/ci/worker/init.sh @@ -8,11 +8,13 @@ export RUNNER_HOME=/home/ubuntu/actions-runner echo "Receiving token" export RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` export RUNNER_URL="https://github.com/ClickHouse" +# Funny fact, but metadata service has fixed IP +export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` cd $RUNNER_HOME echo "Going to configure runner" -sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name `hostname -f` --runnergroup Default --labels 'self-hosted,Linux,X64' --work _work +sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64' --work _work echo "Run" sudo -u ubuntu ./run.sh From 0085e5653a81b6689dc1e1977b4ed421f36279a5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 12:00:45 +0300 Subject: [PATCH 208/950] Metrics lambda --- tests/ci/metrics_lambda/Dockerfile | 13 +++ tests/ci/metrics_lambda/app.py | 138 +++++++++++++++++++++++ tests/ci/metrics_lambda/requirements.txt | 3 + 3 files changed, 154 insertions(+) create mode 100644 tests/ci/metrics_lambda/Dockerfile create mode 100644 tests/ci/metrics_lambda/app.py create mode 100644 tests/ci/metrics_lambda/requirements.txt diff --git a/tests/ci/metrics_lambda/Dockerfile b/tests/ci/metrics_lambda/Dockerfile new file mode 100644 index 00000000000..f53be71a893 --- /dev/null +++ b/tests/ci/metrics_lambda/Dockerfile @@ -0,0 +1,13 @@ +FROM public.ecr.aws/lambda/python:3.9 + +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + +# Install the function's dependencies using file requirements.txt +# from your project folder. + +COPY requirements.txt . +RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" + +# Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) +CMD [ "app.handler" ] diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py new file mode 100644 index 00000000000..8002e060dd0 --- /dev/null +++ b/tests/ci/metrics_lambda/app.py @@ -0,0 +1,138 @@ +#!/usr/bin/env python3 + +import requests +import argparse +import jwt +import sys +import json +import time +from collections import namedtuple + +def get_key_and_app_from_aws(): + import boto3 + secret_name = "clickhouse_github_secret_key_1" + session = boto3.session.Session() + client = session.client( + service_name='secretsmanager', + ) + get_secret_value_response = client.get_secret_value( + SecretId=secret_name + ) + data = json.loads(get_secret_value_response['SecretString']) + return data['clickhouse-app-key'], int(data['clickhouse-app-id']) + +def handler(event, context): + private_key, app_id = get_key_and_app_from_aws() + main(private_key, app_id, True) + +def get_installation_id(jwt_token): + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get("https://api.github.com/app/installations", headers=headers) + response.raise_for_status() + data = response.json() + return data[0]['id'] + +def get_access_token(jwt_token, installation_id): + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.post(f"https://api.github.com/app/installations/{installation_id}/access_tokens", headers=headers) + response.raise_for_status() + data = response.json() + return data['token'] + + +RunnerDescription = namedtuple('RunnerDescription', ['id', 'name', 'tags', 'offline', 'busy']) + +def list_runners(access_token): + headers = { + "Authorization": f"token {access_token}", + "Accept": "application/vnd.github.v3+json", + } + + response = requests.get("https://api.github.com/orgs/ClickHouse/actions/runners", headers=headers) + response.raise_for_status() + data = response.json() + print("Total runners", data['total_count']) + runners = data['runners'] + result = [] + for runner in runners: + tags = [tag['name'] for tag in runner['labels']] + desc = RunnerDescription(id=runner['id'], name=runner['name'], tags=tags, + offline=runner['status']=='offline', busy=runner['busy']) + result.append(desc) + return result + +def push_metrics_to_cloudwatch(listed_runners, namespace): + import boto3 + client = boto3.client('cloudwatch') + metrics_data = [] + busy_runners = sum(1 for runner in listed_runners if runner.busy) + metrics_data.append({ + 'MetricName': 'BusyRunners', + 'Value': busy_runners, + 'Unit': 'Count', + }) + total_active_runners = sum(1 for runner in listed_runners if not runner.offline) + metrics_data.append({ + 'MetricName': 'ActiveRunners', + 'Value': total_active_runners, + 'Unit': 'Count', + }) + total_runners = len(listed_runners) + metrics_data.append({ + 'MetricName': 'TotalRunners', + 'Value': total_runners, + 'Unit': 'Count', + }) + metrics_data.append({ + 'MetricName': 'BusyRunnersRatio', + 'Value': busy_runners / total_active_runners * 100, + 'Unit': 'Percent', + }) + + client.put_metric_data(Namespace='RunnersMetrics', MetricData=metrics_data) + +def main(github_secret_key, github_app_id, push_to_cloudwatch): + payload = { + "iat": int(time.time()) - 60, + "exp": int(time.time()) + (10 * 60), + "iss": github_app_id, + } + + encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256") + installation_id = get_installation_id(encoded_jwt) + access_token = get_access_token(encoded_jwt, installation_id) + runners = list_runners(access_token) + if push_to_cloudwatch: + push_metrics_to_cloudwatch(runners, 'RunnersMetrics') + else: + print(runners) + + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description='Get list of runners and their states') + parser.add_argument('-p', '--private-key-path', help='Path to file with private key') + parser.add_argument('-k', '--private-key', help='Private key') + parser.add_argument('-a', '--app-id', type=int, help='GitHub application ID', required=True) + parser.add_argument('--push-to-cloudwatch', action='store_true', help='Store received token in parameter store') + + args = parser.parse_args() + + if not args.private_key_path and not args.private_key: + print("Either --private-key-path or --private-key must be specified", file=sys.stderr) + + if args.private_key_path and args.private_key: + print("Either --private-key-path or --private-key must be specified", file=sys.stderr) + + if args.private_key: + private_key = args.private_key + else: + with open(args.private_key_path, 'r') as key_file: + private_key = key_file.read() + + main(private_key, args.app_id, args.push_to_cloudwatch) diff --git a/tests/ci/metrics_lambda/requirements.txt b/tests/ci/metrics_lambda/requirements.txt new file mode 100644 index 00000000000..c0dcf4a4dde --- /dev/null +++ b/tests/ci/metrics_lambda/requirements.txt @@ -0,0 +1,3 @@ +requests +PyJWT +cryptography From e53a48fb3089ab114b8fa907fc2efa3e27a1c960 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 30 Sep 2021 10:15:44 +0200 Subject: [PATCH 209/950] Raise lock acquire timeout for the test Needed for check test under ASAN --- tests/queries/0_stateless/02033_join_engine_deadlock.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.sh b/tests/queries/0_stateless/02033_join_engine_deadlock.sh index 7a4ca1c8bb1..f4ae564e2a7 100755 --- a/tests/queries/0_stateless/02033_join_engine_deadlock.sh +++ b/tests/queries/0_stateless/02033_join_engine_deadlock.sh @@ -30,7 +30,7 @@ populate_table_bg () { INSERT INTO join_block_test SELECT toString(number) as id, number * number as num FROM system.numbers LIMIT 3000000 - " --lock_acquire_timeout=20 >/dev/null + " >/dev/null ) & } @@ -54,7 +54,7 @@ read_table_bg () { FROM join_block_test ) AS i2 ON i1.id = toString(i2.num) ) AS t2 ON t1.user_id = t2.id - " --lock_acquire_timeout=20 >/dev/null + " >/dev/null ) & } From f58742014c0a72e9d02d9c02d4172b3d36989461 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 30 Sep 2021 10:47:15 +0200 Subject: [PATCH 210/950] Consistent naming --- src/Storages/StorageJoin.cpp | 27 +++++++++++++-------------- src/Storages/StorageJoin.h | 6 +++--- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index e5574708de0..b17315106ce 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -67,11 +67,11 @@ StorageJoin::StorageJoin( restore(); } -RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const +RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const { - const String query_id = ctx ? ctx->getInitialQueryId() : RWLockImpl::NO_QUERY; + const String query_id = context ? context->getInitialQueryId() : RWLockImpl::NO_QUERY; const std::chrono::milliseconds acquire_timeout - = ctx ? ctx->getSettingsRef().lock_acquire_timeout : std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC); + = context ? context->getSettingsRef().lock_acquire_timeout : std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC); return tryLockTimed(lock, type, query_id, acquire_timeout); } @@ -81,11 +81,10 @@ SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataP return StorageSetOrJoinBase::write(query, metadata_snapshot, context); } -void StorageJoin::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx, TableExclusiveLockHolder&) +void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, TableExclusiveLockHolder &) { std::lock_guard mutate_lock(mutate_mutex); - TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context); disk->removeRecursive(path); disk->createDirectories(path); @@ -160,7 +159,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) } } -HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const +HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context) const { auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) @@ -180,7 +179,7 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns()); - RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx); + RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context); join_clone->setLock(holder); join_clone->reuseJoinedData(*join); @@ -188,15 +187,15 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, } -void StorageJoin::insertBlock(const Block & block, ContextPtr ctx) +void StorageJoin::insertBlock(const Block & block, ContextPtr context) { - TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context); join->addJoinedBlock(block, true); } -size_t StorageJoin::getSize(ContextPtr ctx) const +size_t StorageJoin::getSize(ContextPtr context) const { - TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context); return join->getTotalRowCount(); } @@ -217,9 +216,9 @@ DataTypePtr StorageJoin::joinGetCheckAndGetReturnType(const DataTypes & data_typ return join->joinGetCheckAndGetReturnType(data_types, column_name, or_null); } -ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr ctx) const +ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr context) const { - TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx); + TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context); return join->joinGet(block, block_with_columns_to_add); } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 4926194433c..cdc47531999 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -37,7 +37,7 @@ public: /// Return instance of HashJoin holding lock that protects from insertions to StorageJoin. /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. - HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const; + HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context) const; /// Get result type for function "joinGet(OrNull)" DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; @@ -78,10 +78,10 @@ private: mutable RWLock rwlock = RWLockImpl::create(); mutable std::mutex mutate_mutex; - void insertBlock(const Block & block, ContextPtr ctx) override; + void insertBlock(const Block & block, ContextPtr context) override; void finishInsert() override {} size_t getSize(ContextPtr context) const override; - RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const; + RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const; protected: StorageJoin( From 7d028c3a90aacf1ae3817ea87597036a512ec11a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 13:12:58 +0300 Subject: [PATCH 211/950] Add termination lambda --- tests/ci/metrics_lambda/app.py | 7 +- tests/ci/termination_lambda/Dockerfile | 13 ++ tests/ci/termination_lambda/app.py | 230 +++++++++++++++++++ tests/ci/termination_lambda/requirements.txt | 3 + 4 files changed, 252 insertions(+), 1 deletion(-) create mode 100644 tests/ci/termination_lambda/Dockerfile create mode 100644 tests/ci/termination_lambda/app.py create mode 100644 tests/ci/termination_lambda/requirements.txt diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index 8002e060dd0..6c6fc594847 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -89,9 +89,14 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): 'Value': total_runners, 'Unit': 'Count', }) + if total_active_runners == 0: + busy_ratio = 100 + else: + busy_ratio = busy_runners / total_active_runners * 100 + metrics_data.append({ 'MetricName': 'BusyRunnersRatio', - 'Value': busy_runners / total_active_runners * 100, + 'Value': busy_ratio, 'Unit': 'Percent', }) diff --git a/tests/ci/termination_lambda/Dockerfile b/tests/ci/termination_lambda/Dockerfile new file mode 100644 index 00000000000..f53be71a893 --- /dev/null +++ b/tests/ci/termination_lambda/Dockerfile @@ -0,0 +1,13 @@ +FROM public.ecr.aws/lambda/python:3.9 + +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + +# Install the function's dependencies using file requirements.txt +# from your project folder. + +COPY requirements.txt . +RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" + +# Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) +CMD [ "app.handler" ] diff --git a/tests/ci/termination_lambda/app.py b/tests/ci/termination_lambda/app.py new file mode 100644 index 00000000000..414ad0a0d0f --- /dev/null +++ b/tests/ci/termination_lambda/app.py @@ -0,0 +1,230 @@ +#!/usr/bin/env python3 + +import requests +import argparse +import jwt +import sys +import json +import time +from collections import namedtuple + +def get_key_and_app_from_aws(): + import boto3 + secret_name = "clickhouse_github_secret_key_1" + session = boto3.session.Session() + client = session.client( + service_name='secretsmanager', + ) + get_secret_value_response = client.get_secret_value( + SecretId=secret_name + ) + data = json.loads(get_secret_value_response['SecretString']) + return data['clickhouse-app-key'], int(data['clickhouse-app-id']) + +def get_installation_id(jwt_token): + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get("https://api.github.com/app/installations", headers=headers) + response.raise_for_status() + data = response.json() + return data[0]['id'] + +def get_access_token(jwt_token, installation_id): + headers = { + "Authorization": f"Bearer {jwt_token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.post(f"https://api.github.com/app/installations/{installation_id}/access_tokens", headers=headers) + response.raise_for_status() + data = response.json() + return data['token'] + + +RunnerDescription = namedtuple('RunnerDescription', ['id', 'name', 'tags', 'offline', 'busy']) + +def list_runners(access_token): + headers = { + "Authorization": f"token {access_token}", + "Accept": "application/vnd.github.v3+json", + } + + response = requests.get("https://api.github.com/orgs/ClickHouse/actions/runners", headers=headers) + response.raise_for_status() + data = response.json() + print("Total runners", data['total_count']) + runners = data['runners'] + result = [] + for runner in runners: + tags = [tag['name'] for tag in runner['labels']] + desc = RunnerDescription(id=runner['id'], name=runner['name'], tags=tags, + offline=runner['status']=='offline', busy=runner['busy']) + result.append(desc) + return result + +def how_many_instances_to_kill(event_data): + data_array = event_data['CapacityToTerminate'] + to_kill_by_zone = {} + for av_zone in data_array: + zone_name = av_zone['AvailabilityZone'] + to_kill = av_zone['Capacity'] + if zone_name not in to_kill_by_zone: + to_kill_by_zone[zone_name] = 0 + + to_kill_by_zone[zone_name] += to_kill + return to_kill_by_zone + +def get_candidates_to_be_killed(event_data): + data_array = event_data['Instances'] + instances_by_zone = {} + for instance in data_array: + zone_name = instance['AvailabilityZone'] + instance_id = instance['InstanceId'] + if zone_name not in instances_by_zone: + instances_by_zone[zone_name] = [] + instances_by_zone[zone_name].append(instance_id) + + return instances_by_zone + +def delete_runner(access_token, runner): + headers = { + "Authorization": f"token {access_token}", + "Accept": "application/vnd.github.v3+json", + } + + response = requests.delete(f"https://api.github.com/orgs/ClickHouse/actions/runners/{runner.id}", headers=headers) + response.raise_for_status() + print(f"Response code deleting {runner.name} is {response.status_code}") + return response.status_code == 204 + + +def main(github_secret_key, github_app_id, event): + print("Got event", json.dumps(event, sort_keys=True, indent=4)) + to_kill_by_zone = how_many_instances_to_kill(event) + instances_by_zone = get_candidates_to_be_killed(event) + + payload = { + "iat": int(time.time()) - 60, + "exp": int(time.time()) + (10 * 60), + "iss": github_app_id, + } + + encoded_jwt = jwt.encode(payload, github_secret_key, algorithm="RS256") + installation_id = get_installation_id(encoded_jwt) + access_token = get_access_token(encoded_jwt, installation_id) + + runners = list_runners(access_token) + + to_delete_runners = [] + instances_to_kill = [] + for zone in to_kill_by_zone: + num_to_kill = to_kill_by_zone[zone] + candidates = instances_by_zone[zone] + if num_to_kill > len(candidates): + raise Exception(f"Required to kill {num_to_kill}, but have only {len(candidates)} candidates in AV {zone}") + + delete_for_av = [] + for candidate in candidates: + if len(delete_for_av) == num_to_kill: + break + for runner in runners: + if runner.name == candidate: + if not runner.busy: + print(f"Runner {runner.name} is not busy and can be deleted from AV {zone}") + delete_for_av.append(runner) + else: + print(f"Runner {runner.name} is busy, not going to delete it") + break + else: + print(f"Candidate {candidate} was not in runners list, simply delete it") + instances_to_kill.append(candidate) + + if len(delete_for_av) < num_to_kill: + print(f"Checked all candidates for av {zone}, get to delete {len(delete_for_av)}, but still cannot get required {num_to_kill}") + to_delete_runners += delete_for_av + + print("Got instances to kill: ", ', '.join(instances_to_kill)) + print("Going to delete runners:", ', '.join([runner.name for runner in to_delete_runners])) + for runner in to_delete_runners: + if delete_runner(access_token, runner): + print(f"Runner {runner.name} successfuly deleted from github") + instances_to_kill.append(runner.name) + else: + print(f"Cannot delete {runner.name} from github") + + response = { + "InstanceIDs": instances_to_kill + } + print(response) + return response + +def handler(event, context): + private_key, app_id = get_key_and_app_from_aws() + return main(private_key, app_id, event) + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description='Get list of runners and their states') + parser.add_argument('-p', '--private-key-path', help='Path to file with private key') + parser.add_argument('-k', '--private-key', help='Private key') + parser.add_argument('-a', '--app-id', type=int, help='GitHub application ID', required=True) + + args = parser.parse_args() + + if not args.private_key_path and not args.private_key: + print("Either --private-key-path or --private-key must be specified", file=sys.stderr) + + if args.private_key_path and args.private_key: + print("Either --private-key-path or --private-key must be specified", file=sys.stderr) + + if args.private_key: + private_key = args.private_key + else: + with open(args.private_key_path, 'r') as key_file: + private_key = key_file.read() + + sample_event = { + "AutoScalingGroupARN": "arn:aws:autoscaling:us-east-1::autoScalingGroup:d4738357-2d40-4038-ae7e-b00ae0227003:autoScalingGroupName/my-asg", + "AutoScalingGroupName": "my-asg", + "CapacityToTerminate": [ + { + "AvailabilityZone": "us-east-1b", + "Capacity": 1, + "InstanceMarketOption": "OnDemand" + }, + { + "AvailabilityZone": "us-east-1c", + "Capacity": 2, + "InstanceMarketOption": "OnDemand" + } + ], + "Instances": [ + { + "AvailabilityZone": "us-east-1b", + "InstanceId": "i-08d0b3c1a137e02a5", + "InstanceType": "t2.nano", + "InstanceMarketOption": "OnDemand" + }, + { + "AvailabilityZone": "us-east-1c", + "InstanceId": "ip-172-31-45-253.eu-west-1.compute.internal", + "InstanceType": "t2.nano", + "InstanceMarketOption": "OnDemand" + }, + { + "AvailabilityZone": "us-east-1c", + "InstanceId": "ip-172-31-27-227.eu-west-1.compute.internal", + "InstanceType": "t2.nano", + "InstanceMarketOption": "OnDemand" + }, + { + "AvailabilityZone": "us-east-1c", + "InstanceId": "ip-172-31-45-253.eu-west-1.compute.internal", + "InstanceType": "t2.nano", + "InstanceMarketOption": "OnDemand" + }, + ], + "Cause": "SCALE_IN" + } + + main(private_key, args.app_id, sample_event) diff --git a/tests/ci/termination_lambda/requirements.txt b/tests/ci/termination_lambda/requirements.txt new file mode 100644 index 00000000000..c0dcf4a4dde --- /dev/null +++ b/tests/ci/termination_lambda/requirements.txt @@ -0,0 +1,3 @@ +requests +PyJWT +cryptography From 8a9556dd9367544e0b6185e5ae71babf987eaa7f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 13:39:15 +0300 Subject: [PATCH 212/950] Update termination lambda --- tests/ci/termination_lambda/app.py | 40 ++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/tests/ci/termination_lambda/app.py b/tests/ci/termination_lambda/app.py index 414ad0a0d0f..261403dd8be 100644 --- a/tests/ci/termination_lambda/app.py +++ b/tests/ci/termination_lambda/app.py @@ -63,6 +63,42 @@ def list_runners(access_token): result.append(desc) return result +def push_metrics_to_cloudwatch(listed_runners, namespace): + import boto3 + client = boto3.client('cloudwatch') + metrics_data = [] + busy_runners = sum(1 for runner in listed_runners if runner.busy) + metrics_data.append({ + 'MetricName': 'BusyRunners', + 'Value': busy_runners, + 'Unit': 'Count', + }) + total_active_runners = sum(1 for runner in listed_runners if not runner.offline) + metrics_data.append({ + 'MetricName': 'ActiveRunners', + 'Value': total_active_runners, + 'Unit': 'Count', + }) + total_runners = len(listed_runners) + metrics_data.append({ + 'MetricName': 'TotalRunners', + 'Value': total_runners, + 'Unit': 'Count', + }) + if total_active_runners == 0: + busy_ratio = 100 + else: + busy_ratio = busy_runners / total_active_runners * 100 + + metrics_data.append({ + 'MetricName': 'BusyRunnersRatio', + 'Value': busy_ratio, + 'Unit': 'Percent', + }) + + client.put_metric_data(Namespace='RunnersMetrics', MetricData=metrics_data) + + def how_many_instances_to_kill(event_data): data_array = event_data['CapacityToTerminate'] to_kill_by_zone = {} @@ -153,6 +189,10 @@ def main(github_secret_key, github_app_id, event): else: print(f"Cannot delete {runner.name} from github") + # push metrics + runners = list_runners(access_token) + push_metrics_to_cloudwatch(runners, 'RunnersMetrics') + response = { "InstanceIDs": instances_to_kill } From f2837569f57712a3d6edd849748ba4028bd4f4c3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 14:26:46 +0300 Subject: [PATCH 213/950] Fixes in termination lambda --- tests/ci/run_check.py | 5 ++++- tests/ci/termination_lambda/app.py | 21 +++++++++++++-------- 2 files changed, 17 insertions(+), 9 deletions(-) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 70b3ae2ac07..95e827671ca 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -86,7 +86,7 @@ def pr_is_by_trusted_user(pr_user_login, pr_user_orgs): # can be skipped entirely. def should_run_checks_for_pr(pr_info): # Consider the labels and whether the user is trusted. - force_labels = set(['force tests', 'release']).intersection(pr_info.labels) + force_labels = set(['force tests']).intersection(pr_info.labels) if force_labels: return True, "Labeled '{}'".format(', '.join(force_labels)) @@ -96,6 +96,9 @@ def should_run_checks_for_pr(pr_info): if 'can be tested' not in pr_info.labels and not pr_is_by_trusted_user(pr_info.user_login, pr_info.user_orgs): return False, "Needs 'can be tested' label" + if 'release' in pr_info.labels or 'pr-backport' in pr_info.labels or 'pr-cherrypick' in pr_info.labels: + return False, "Don't try new checks for release/backports/cherry-picks" + return True, "No special conditions apply" def get_commit(gh, commit_sha): diff --git a/tests/ci/termination_lambda/app.py b/tests/ci/termination_lambda/app.py index 261403dd8be..7fd7c400db9 100644 --- a/tests/ci/termination_lambda/app.py +++ b/tests/ci/termination_lambda/app.py @@ -162,8 +162,16 @@ def main(github_secret_key, github_app_id, event): delete_for_av = [] for candidate in candidates: - if len(delete_for_av) == num_to_kill: + if candidate not in set([runner.name for runner in runners]): + print(f"Candidate {candidate} was not in runners list, simply delete it") + instances_to_kill.append(candidate) + + for candidate in candidates: + if len(delete_for_av) + len(instances_to_kill) == num_to_kill: break + if candidate in instances_to_kill: + continue + for runner in runners: if runner.name == candidate: if not runner.busy: @@ -172,9 +180,6 @@ def main(github_secret_key, github_app_id, event): else: print(f"Runner {runner.name} is busy, not going to delete it") break - else: - print(f"Candidate {candidate} was not in runners list, simply delete it") - instances_to_kill.append(candidate) if len(delete_for_av) < num_to_kill: print(f"Checked all candidates for av {zone}, get to delete {len(delete_for_av)}, but still cannot get required {num_to_kill}") @@ -189,9 +194,9 @@ def main(github_secret_key, github_app_id, event): else: print(f"Cannot delete {runner.name} from github") - # push metrics - runners = list_runners(access_token) - push_metrics_to_cloudwatch(runners, 'RunnersMetrics') + ## push metrics + #runners = list_runners(access_token) + #push_metrics_to_cloudwatch(runners, 'RunnersMetrics') response = { "InstanceIDs": instances_to_kill @@ -262,7 +267,7 @@ if __name__ == "__main__": "InstanceId": "ip-172-31-45-253.eu-west-1.compute.internal", "InstanceType": "t2.nano", "InstanceMarketOption": "OnDemand" - }, + } ], "Cause": "SCALE_IN" } From 7d92ad66149daac84601a69cde38b03f78668db0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 15:09:06 +0300 Subject: [PATCH 214/950] Remove PVS check --- .github/workflows/main.yml | 22 ---------------------- 1 file changed, 22 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 8cb771a0d45..49760995dfc 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -54,28 +54,6 @@ jobs: YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py -# PVS-Check: -# needs: DockerHubPush -# runs-on: [self-hosted] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/pvs_check -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# - name: PVS Check -# env: -# YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} -# YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} -# GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} -# PVS_STUDIO_KEY: ${{ secrets.PVS_STUDIO_KEY }} -# TEMP_PATH: ${{runner.temp}}/pvs_check -# REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse -# run: mkdir -p ${{runner.temp}}/pvs_check && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 pvs_check.py FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels] runs-on: [self-hosted] From 2aa852388fa2e372326603ee78ec4ab04ee72e72 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 15:34:44 +0300 Subject: [PATCH 215/950] Fix style check --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index efdc5f488d2..b2334a8b203 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -162,7 +162,7 @@ find $ROOT_PATH -name '.gitmodules' | while read i; do grep -F 'url = ' $i | gre find $ROOT_PATH/{src,base,programs} -name '*.h' -or -name '*.cpp' 2>/dev/null | xargs grep -i -F 'General Public License' && echo "There shouldn't be any code snippets under GPL or LGPL" # There shouldn't be any docker containers outside docker directory -find $ROOT_PATH -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:" +find $ROOT_PATH -not -path $ROOT_PATH'/tests/ci*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:" # There shouldn't be any docker compose files outside docker directory #find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:" From 1df70af14e2de27405f6cfdacc651567b6140684 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 16:08:12 +0300 Subject: [PATCH 216/950] Fix style check one more time: --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index b2334a8b203..dc954411918 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -70,7 +70,7 @@ find $ROOT_PATH/{src,base,programs,utils} -name '*.xml' | xargs xmllint --noout --nonet # FIXME: for now only clickhouse-test -pylint --rcfile=$ROOT_PATH/.pylintrc --score=n $ROOT_PATH/tests/clickhouse-test +pylint --rcfile=$ROOT_PATH/.pylintrc --persistent=no --score=n $ROOT_PATH/tests/clickhouse-test find $ROOT_PATH -not -path $ROOT_PATH'/contrib*' \( -name '*.yaml' -or -name '*.yml' \) -type f | grep -vP $EXCLUDE_DIRS | From 7a27ce7242abd679650b227051f7598bda854ecd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 16:20:08 +0300 Subject: [PATCH 217/950] Pull new image each time --- tests/ci/style_check.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 2af8514fbfc..71978379099 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -121,6 +121,15 @@ if __name__ == "__main__": docker_image += ':' + images['clickhouse/style-test'] logging.info("Got docker image %s", docker_image) + for i in range(10): + try: + subprocess.check_output(f"docker pull {docker_image}", shell=True) + break + except Exception as ex: + time.sleep(i * 3) + logging.info("Got execption pulling docker %s", ex) + else: + raise Exception(f"Cannot pull dockerhub for image {docker_image}") if not aws_secret_key_id or not aws_secret_key: logging.info("No secrets, will not upload anything to S3") From cfb9875acf3df38fe3f28188a0c79aff7a3e7a97 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Thu, 30 Sep 2021 13:47:12 +0000 Subject: [PATCH 218/950] Trigger Build From 6f2447c027526d06e6f0125ea496815f01d40052 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 30 Sep 2021 15:48:54 +0200 Subject: [PATCH 219/950] clang-tidy fix --- src/Storages/StorageSet.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index c57dadf6d52..fd06c2975b6 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -106,10 +106,11 @@ void SetOrJoinSink::onFinish() } -SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx) +SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { UInt64 id = ++increment; - return std::make_shared(ctx, *this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent); + return std::make_shared( + context, *this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent); } From 1758ff2d321d3cd7b0cc1b3b4d785a8e389e53d4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 30 Sep 2021 16:02:17 +0000 Subject: [PATCH 220/950] fix --- src/Storages/FileLog/FileLogSettings.h | 8 +- src/Storages/FileLog/FileLogSource.cpp | 40 +- src/Storages/FileLog/FileLogSource.h | 3 +- .../FileLog/ReadBufferFromFileLog.cpp | 33 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 4 - src/Storages/FileLog/StorageFileLog.cpp | 356 ++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 39 +- 7 files changed, 275 insertions(+), 208 deletions(-) diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index ba76b4d02da..5ed55d94426 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -11,10 +11,10 @@ class ASTStorage; #define FILELOG_RELATED_SETTINGS(M) \ /* default is stream_poll_timeout_ms */ \ - M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \ - M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single filelog poll.", 0) \ - M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from filelog.", 0) \ - M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0) + M(Milliseconds, poll_timeout_ms, 0, "Timeout for single poll from StorageFileLog.", 0) \ + M(UInt64, poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single StorageFileLog poll.", 0) \ + M(UInt64, max_block_size, 0, "Number of row collected by poll(s) for flushing data from StorageFileLog.", 0) \ + M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0) #define LIST_OF_FILELOG_SETTINGS(M) \ FILELOG_RELATED_SETTINGS(M) \ diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 590630658ca..e2223b3d3c2 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -17,14 +17,16 @@ FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, + const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : SourceWithProgress(metadata_snapshot_->getSampleBlockWithVirtuals(storage_.getVirtuals())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) + , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) , stream_number(stream_number_) @@ -34,19 +36,20 @@ FileLogSource::FileLogSource( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); - /// The last FileLogSource responsible for open files - if (stream_number == max_streams_number - 1) - { - storage.openFilesAndSetPos(); - } } FileLogSource::~FileLogSource() { - /// The last FileLogSource responsible for close files - if (stream_number == max_streams_number - 1) + auto & file_infos = storage.getFileInfos(); + + size_t files_per_stream = file_infos.file_names.size() / max_streams_number; + size_t start = stream_number * files_per_stream; + size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; + + /// Each stream responsible for close it's files and store meta + for (size_t i = start; i < end; ++i) { - storage.closeFilesAndStoreMeta(); + storage.closeFileAndStoreMeta(file_infos.file_names[i]); } } @@ -99,14 +102,21 @@ Chunk FileLogSource::generate() if (total_rows == 0) return {}; - auto result_columns = executor.getResultColumns(); + auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - for (auto & column : virtual_columns) - { - result_columns.emplace_back(std::move(column)); - } + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); - return Chunk(std::move(result_columns), total_rows); + auto converting_dag = ActionsDAG::makeConvertingActions( + result_block.cloneEmpty().getColumnsWithTypeAndName(), + getPort().getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(result_block); + + return Chunk(result_block.getColumns(), result_block.rows()); } } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 115c68f2e25..d257bb54f06 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -5,7 +5,6 @@ #include #include - namespace Poco { class Logger; @@ -19,6 +18,7 @@ public: StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, + const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, @@ -37,6 +37,7 @@ private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; ContextPtr context; + Names column_names; UInt64 max_block_size; size_t poll_time_out; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index e39fd84acfe..e161c69a960 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -31,15 +31,8 @@ ReadBufferFromFileLog::ReadBufferFromFileLog( , stream_number(stream_number_) , max_streams_number(max_streams_number_) { - cleanUnprocessed(); - allowed = false; -} - -void ReadBufferFromFileLog::cleanUnprocessed() -{ - records.clear(); current = records.begin(); - BufferBase::set(nullptr, 0, 0); + allowed = false; } bool ReadBufferFromFileLog::poll() @@ -50,11 +43,10 @@ bool ReadBufferFromFileLog::poll() return true; } - buffer_status = BufferStatus::NO_RECORD_RETURNED; - auto new_records = pollBatch(batch_size); if (new_records.empty()) { + buffer_status = BufferStatus::NO_RECORD_RETURNED; LOG_TRACE(log, "No records returned"); return false; } @@ -106,33 +98,28 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ for (size_t i = start; i < end; ++i) { - auto file_name = file_infos.file_names[i]; - auto & file_ctx = file_infos.context_by_name.at(file_name); + const auto & file_name = file_infos.file_names[i]; + + auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name); if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) continue; - auto & file_meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file_name)); + auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); Record record; while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) { - if (!file_ctx.reader.good()) - { - throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - UInt64 start_offset = file_ctx.reader.tellg(); + StorageFileLog::assertStreamGood(file_ctx.reader); + std::getline(file_ctx.reader, record.data); record.file_name = file_name; - record.offset = start_offset; + record.offset = file_ctx.reader.tellg(); new_records.emplace_back(record); ++read_records_size; } UInt64 current_position = file_ctx.reader.tellg(); - if (!file_ctx.reader.good()) - { - throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } + StorageFileLog::assertStreamGood(file_ctx.reader); file_meta.last_writen_position = current_position; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 8a3fdf235d7..c792978a68a 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -83,14 +83,10 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; - TaskThread wait_task; - Records pollBatch(size_t batch_size_); void readNewRecords(Records & new_records, size_t batch_size_); - void cleanUnprocessed(); - bool nextImpl() override; }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index d5d86eec0b8..18a76b72af8 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -19,9 +20,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -41,6 +42,7 @@ namespace ErrorCodes extern const int READ_META_FILE_FAILED; extern const int FILE_STREAM_ERROR; extern const int LOGICAL_ERROR; + extern const int TABLE_METADATA_ALREADY_EXISTS; } namespace @@ -54,20 +56,24 @@ StorageFileLog::StorageFileLog( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & relative_path_, + const String & path_, + const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, + const String & comment, bool attach) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) - , path(getContext()->getUserFilesPath() + "/" + relative_path_) + , path(path_) + , relative_data_path(relative_data_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) , milliseconds_to_wait(RESCHEDULE_MS) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); try @@ -75,9 +81,11 @@ StorageFileLog::StorageFileLog( loadMetaFiles(attach); loadFiles(); +#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); +#endif if (path_is_directory) directory_watch = std::make_unique(path); @@ -93,17 +101,20 @@ StorageFileLog::StorageFileLog( void StorageFileLog::loadMetaFiles(bool attach) { - const auto database = DatabaseCatalog::instance().getDatabase(getStorageID().getDatabaseName()); - const auto table_name = getStorageID().getTableName(); - - root_meta_path = database->getMetadataPath() + "/." + table_name; + /// We just use default storage policy + auto storage_policy = getContext()->getStoragePolicy("default"); + auto data_volume = storage_policy->getVolume(0); + root_meta_path = std::filesystem::path(data_volume->getDisk()->getPath()) / getStorageID().getTableName(); /// Create table, just create meta data directory if (!attach) { if (std::filesystem::exists(root_meta_path)) { - std::filesystem::remove_all(root_meta_path); + throw Exception( + ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, + "Metadata files already exist by path: {}, remove them manually if it is intended", + root_meta_path); } std::filesystem::create_directories(root_meta_path); } @@ -113,7 +124,7 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Meta file may lost, log and create directory if (!std::filesystem::exists(root_meta_path)) { - LOG_INFO(log, "Meta files of table {} may have lost.", getStorageID().getTableName()); + LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName()); std::filesystem::create_directories(root_meta_path); } /// Load all meta info to file_infos; @@ -123,19 +134,27 @@ void StorageFileLog::loadMetaFiles(bool attach) void StorageFileLog::loadFiles() { + if (!symlinkStartsWith(path, getContext()->getUserFilesPath())) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The absolute data path should start with user_files_path {}", getContext()->getUserFilesPath()); + } - if (std::filesystem::is_regular_file(path)) + auto absolute_path = std::filesystem::absolute(path); + absolute_path = absolute_path.lexically_normal(); /// Normalize path. + + if (std::filesystem::is_regular_file(absolute_path)) { path_is_directory = false; - root_data_path = getContext()->getUserFilesPath(); + root_data_path = absolute_path.parent_path(); - file_infos.file_names.push_back(std::filesystem::path(path).filename()); + file_infos.file_names.push_back(absolute_path.filename()); } - else if (std::filesystem::is_directory(path)) + else if (std::filesystem::is_directory(absolute_path)) { - root_data_path = path; + root_data_path = absolute_path; /// Just consider file with depth 1 - for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + for (const auto & dir_entry : std::filesystem::directory_iterator{absolute_path}) { if (dir_entry.is_regular_file()) { @@ -145,76 +164,86 @@ void StorageFileLog::loadFiles() } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", absolute_path.c_str()); } /// Get files inode for (const auto & file : file_infos.file_names) { auto inode = getInode(getFullDataPath(file)); - file_infos.inode_by_name.emplace(file, inode); - file_infos.context_by_name.emplace(file, FileContext{}); + file_infos.context_by_name.emplace(file, FileContext{.inode = inode}); } /// Update file meta or create file meta - for (const auto & file_inode : file_infos.inode_by_name) + for (const auto & [file, ctx] : file_infos.context_by_name) { - if (auto it = file_infos.meta_by_inode.find(file_inode.second); it != file_infos.meta_by_inode.end()) + if (auto it = file_infos.meta_by_inode.find(ctx.inode); it != file_infos.meta_by_inode.end()) { /// data file have been renamed, need update meta file's name - if (it->second.file_name != file_inode.first) + if (it->second.file_name != file) { - it->second.file_name = file_inode.first; + it->second.file_name = file; if (std::filesystem::exists(getFullMetaPath(it->second.file_name))) { - std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file_inode.first)); + std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); } } } /// New file else { - FileMeta meta{file_inode.first, 0, 0}; - file_infos.meta_by_inode.emplace(file_inode.second, meta); + FileMeta meta{file, 0, 0}; + file_infos.meta_by_inode.emplace(ctx.inode, meta); } } /// Clear unneeded meta file, because data files may be deleted - if (file_infos.meta_by_inode.size() > file_infos.inode_by_name.size()) + if (file_infos.meta_by_inode.size() > file_infos.context_by_name.size()) { InodeToFileMeta valid_metas; - valid_metas.reserve(file_infos.inode_by_name.size()); + valid_metas.reserve(file_infos.context_by_name.size()); for (const auto & it : file_infos.meta_by_inode) { - if (file_infos.inode_by_name.contains(it.second.file_name)) + auto file_name = it.second.file_name; + if (file_infos.context_by_name.contains(file_name)) valid_metas.emplace(it); + /// Delete meta file from filesystem + else + std::filesystem::remove(getFullMetaPath(file_name)); } file_infos.meta_by_inode.swap(valid_metas); } } -void StorageFileLog::serialize(bool with_end_pos) const +void StorageFileLog::serialize() const { for (const auto & it : file_infos.meta_by_inode) { auto full_name = getFullMetaPath(it.second.file_name); if (!std::filesystem::exists(full_name)) { - Poco::File{full_name}.createFile(); - } - WriteBufferFromFile buf(full_name); - writeIntText(it.first, buf); - writeChar('\n', buf); - writeIntText(it.second.last_writen_position, buf); - - if (with_end_pos) - { - writeChar('\n', buf); - writeIntText(it.second.last_open_end, buf); + FS::createFile(full_name); } + WriteBufferFromFile out(full_name); + writeIntText(it.first, out); + writeChar('\n', out); + writeIntText(it.second.last_writen_position, out); } } +void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const +{ + auto full_name = getFullMetaPath(file_meta.file_name); + if (!std::filesystem::exists(full_name)) + { + FS::createFile(full_name); + } + WriteBufferFromFile out(full_name); + writeIntText(inode, out); + writeChar('\n', out); + writeIntText(file_meta.last_writen_position, out); +} + void StorageFileLog::deserialize() { for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) @@ -228,19 +257,16 @@ void StorageFileLog::deserialize() root_meta_path); } - ReadBufferFromFile buf(dir_entry.path().c_str()); + ReadBufferFromFile in(dir_entry.path().c_str()); FileMeta meta; UInt64 inode, last_written_pos; - if (!tryReadIntText(inode, buf)) + if (!tryReadIntText(inode, in)) { throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); } - if (!checkChar('\n', buf)) - { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); - } - if (!tryReadIntText(last_written_pos, buf)) + assertChar('\n', in); + if (!tryReadIntText(last_written_pos, in)) { throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); } @@ -248,15 +274,6 @@ void StorageFileLog::deserialize() meta.file_name = dir_entry.path().filename(); meta.last_writen_position = last_written_pos; - /// May have last open end in meta file - if (checkChar('\n', buf)) - { - if (!tryReadIntText(meta.last_open_end, buf)) - { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); - } - } - file_infos.meta_by_inode.emplace(inode, meta); } } @@ -280,6 +297,7 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { + /// We need this lock, in case read and streamToViews execute at the same time std::lock_guard lock(status_mutex); updateFileInfos(); @@ -287,30 +305,30 @@ Pipe StorageFileLog::read( /// No files to parse if (file_infos.file_names.empty()) { + LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); return Pipe{}; } auto modified_context = Context::createCopy(local_context); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); + auto max_streams_number = std::min(filelog_settings->max_threads, file_infos.file_names.size()); + + /// Each stream responsible for closing it's files and store meta + openFilesAndSetPos(); Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - Pipe pipe(std::make_shared( - *this, metadata_snapshot, modified_context, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); - - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); - pipes.emplace_back(std::move(pipe)); + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + modified_context, + column_names, + getMaxBlockSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); } return Pipe::unitePipes(std::move(pipes)); @@ -318,15 +336,29 @@ Pipe StorageFileLog::read( void StorageFileLog::drop() { - if (std::filesystem::exists(root_meta_path)) - std::filesystem::remove_all(root_meta_path); + try + { + if (std::filesystem::exists(root_meta_path)) + std::filesystem::remove_all(root_meta_path); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void StorageFileLog::startup() { - if (task) + try { - task->holder->activateAndSchedule(); + if (task) + { + task->holder->activateAndSchedule(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -342,32 +374,31 @@ void StorageFileLog::shutdown() closeFilesAndStoreMeta(); } +void StorageFileLog::assertStreamGood(const std::ifstream & reader) +{ + if (!reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Stream is in bad state."); + } +} + void StorageFileLog::openFilesAndSetPos() { for (const auto & file : file_infos.file_names) { - auto & file_ctx = file_infos.context_by_name.at(file); + auto & file_ctx = findInMap(file_infos.context_by_name, file); if (file_ctx.status != FileStatus::NO_CHANGE) { file_ctx.reader.open(getFullDataPath(file)); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); - } + assertStreamGood(file_ctx.reader); file_ctx.reader.seekg(0, file_ctx.reader.end); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); - } + assertStreamGood(file_ctx.reader); auto file_end = file_ctx.reader.tellg(); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file); - } + assertStreamGood(file_ctx.reader); - auto & meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file)); + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); @@ -376,13 +407,10 @@ void StorageFileLog::openFilesAndSetPos() meta.last_open_end = file_end; file_ctx.reader.seekg(meta.last_writen_position); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); - } + assertStreamGood(file_ctx.reader); } } - serialize(true); + serialize(); } void StorageFileLog::closeFilesAndStoreMeta() @@ -395,23 +423,33 @@ void StorageFileLog::closeFilesAndStoreMeta() serialize(); } +void StorageFileLog::closeFileAndStoreMeta(const String & file_name) +{ + auto & file_ctx = findInMap(file_infos.context_by_name, file_name); + if (file_ctx.reader.is_open()) + file_ctx.reader.close(); + + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + serialize(file_ctx.inode, meta); +} + size_t StorageFileLog::getMaxBlockSize() const { - return filelog_settings->filelog_max_block_size.changed ? filelog_settings->filelog_max_block_size.value - : getContext()->getSettingsRef().max_insert_block_size.value; + return filelog_settings->max_block_size.changed ? filelog_settings->max_block_size.value + : getContext()->getSettingsRef().max_insert_block_size.value; } size_t StorageFileLog::getPollMaxBatchSize() const { - size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value - : getContext()->getSettingsRef().max_block_size.value; + size_t batch_size = filelog_settings->poll_max_batch_size.changed ? filelog_settings->poll_max_batch_size.value + : getContext()->getSettingsRef().max_block_size.value; return std::min(batch_size, getMaxBlockSize()); } size_t StorageFileLog::getPollTimeoutMillisecond() const { - return filelog_settings->filelog_poll_timeout_ms.changed ? filelog_settings->filelog_poll_timeout_ms.totalMilliseconds() - : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); + return filelog_settings->poll_timeout_ms.changed ? filelog_settings->poll_timeout_ms.totalMilliseconds() + : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } bool StorageFileLog::checkDependencies(const StorageID & table_id) @@ -503,10 +541,11 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_infos.file_names.size()); + auto max_streams_number = std::min(filelog_settings->max_threads.value, file_infos.file_names.size()); /// No files to parse if (max_streams_number == 0) { + LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); return false; } @@ -519,36 +558,38 @@ bool StorageFileLog::streamToViews() InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); + /// Each stream responsible for closing it's files and store meta + openFilesAndSetPos(); + Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - Pipe pipe(std::make_shared( - *this, metadata_snapshot, new_context, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); - - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - block_io.pipeline.getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); - pipes.emplace_back(std::move(pipe)); + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + new_context, + block_io.pipeline.getHeader().getNames(), + getPollMaxBatchSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); } auto input= Pipe::unitePipes(std::move(pipes)); assertBlocksHaveEqualStructure(input.getHeader(), block_io.pipeline.getHeader(), "StorageFileLog streamToViews"); - block_io.pipeline.complete(std::move(input)); - - CompletedPipelineExecutor executor(block_io.pipeline); - executor.execute(); + size_t rows = 0; + { + block_io.pipeline.complete(std::move(input)); + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } UInt64 milliseconds = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing data to {} took {} ms.", table_id.getNameForLogs(), milliseconds); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", rows, table_id.getNameForLogs(), milliseconds); return updateFileInfos(); } @@ -569,7 +610,7 @@ void registerStorageFileLog(StorageFactory & factory) } auto physical_cpu_cores = getNumberOfPhysicalCPUCores(); - auto num_threads = filelog_settings->filelog_max_threads.value; + auto num_threads = filelog_settings->max_threads.value; if (num_threads > physical_cpu_cores) { @@ -580,12 +621,12 @@ void registerStorageFileLog(StorageFactory & factory) throw Exception("Number of threads to parse files can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } - if (filelog_settings->filelog_max_block_size.changed && filelog_settings->filelog_max_block_size.value < 1) + if (filelog_settings->max_block_size.changed && filelog_settings->max_block_size.value < 1) { throw Exception("filelog_max_block_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } - if (filelog_settings->filelog_poll_max_batch_size.changed && filelog_settings->filelog_poll_max_batch_size.value < 1) + if (filelog_settings->poll_max_batch_size.changed && filelog_settings->poll_max_batch_size.value < 1) { throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } @@ -601,7 +642,15 @@ void registerStorageFileLog(StorageFactory & factory) auto format = format_ast->as().value.safeGet(); return StorageFileLog::create( - args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings), args.attach); + args.table_id, + args.getContext(), + args.columns, + path, + args.relative_data_path, + format, + std::move(filelog_settings), + args.comment, + args.attach); }; factory.registerStorage( @@ -619,9 +668,13 @@ bool StorageFileLog::updateFileInfos() /// For table just watch one file, we can not use directory monitor to watch it if (!path_is_directory) { - assert( - file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() - == file_infos.context_by_name.size() == 1); +#ifndef NDEBUG + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == 1); +#endif + if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end()) { it->second.status = FileStatus::UPDATED; @@ -642,7 +695,8 @@ bool StorageFileLog::updateFileInfos() { switch (event.type) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: { + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + { LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(event.path)) { @@ -650,45 +704,40 @@ bool StorageFileLog::updateFileInfos() auto inode = getInode(event.path); file_infos.file_names.push_back(file_name); - file_infos.inode_by_name.emplace(file_name, inode); - FileMeta meta{file_name, 0, 0}; - file_infos.meta_by_inode.emplace(inode, meta); - file_infos.context_by_name.emplace(file_name, FileContext{}); + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); } break; } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { - auto file_name = std::filesystem::path(event.path).filename(); + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + { + String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::UPDATED; - } + auto & file_ctx = findInMap(file_infos.context_by_name, file_name); + file_ctx.status = FileStatus::UPDATED; break; } case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { - auto file_name = std::filesystem::path(event.path).filename(); + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + { + String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::REMOVED; - } + auto & file_ctx = findInMap(file_infos.context_by_name, file_name); + file_ctx.status = FileStatus::REMOVED; break; } /// file rename - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + { auto file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); file_infos.file_names.push_back(file_name); - file_infos.context_by_name.emplace(file_name, FileContext{}); - auto inode = getInode(event.path); - file_infos.inode_by_name.emplace(file_name, inode); + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { @@ -705,19 +754,18 @@ bool StorageFileLog::updateFileInfos() } std::vector valid_files; + /// Remove file infos with REMOVE status for (const auto & file_name : file_infos.file_names) { if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED) { + /// Erase meta_by_inode first, otherwise it become invalid + file_infos.meta_by_inode.erase(it->second.inode); file_infos.context_by_name.erase(it); - if (auto inode = file_infos.inode_by_name.find(file_name); inode != file_infos.inode_by_name.end()) - { - file_infos.inode_by_name.erase(inode); - file_infos.meta_by_inode.erase(inode->second); - if (std::filesystem::exists(getFullMetaPath(file_name))) - std::filesystem::remove(getFullMetaPath(file_name)); - } + + if (std::filesystem::exists(getFullMetaPath(file_name))) + std::filesystem::remove(getFullMetaPath(file_name)); } else { @@ -727,9 +775,11 @@ bool StorageFileLog::updateFileInfos() file_infos.file_names.swap(valid_files); /// These file infos should always have same size(one for one) +#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); +#endif return events.empty() || file_infos.file_names.empty(); } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ec81558438c..7384543fa36 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -18,6 +18,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; @@ -59,6 +64,7 @@ public: struct FileContext { FileStatus status = FileStatus::OPEN; + UInt64 inode; std::ifstream reader; }; @@ -69,13 +75,11 @@ public: UInt64 last_open_end = 0; }; - using FileNameToInode = std::unordered_map; using InodeToFileMeta = std::unordered_map; using FileNameToContext = std::unordered_map; struct FileInfos { - FileNameToInode inode_by_name; InodeToFileMeta meta_by_inode; FileNameToContext context_by_name; /// file names without path @@ -84,8 +88,8 @@ public: auto & getFileInfos() { return file_infos; } - auto getFullMetaPath(const String & file_name) const { return root_meta_path + "/" + file_name; } - auto getFullDataPath(const String & file_name) const { return root_data_path + "/" + file_name; } + String getFullMetaPath(const String & file_name) const { return std::filesystem::path(root_meta_path) / file_name; } + String getFullDataPath(const String & file_name) const { return std::filesystem::path(root_data_path) / file_name; } NamesAndTypesList getVirtuals() const override; @@ -94,23 +98,40 @@ public: static UInt64 getInode(const String & file_name); void openFilesAndSetPos(); + /// Used in shutdown() void closeFilesAndStoreMeta(); + /// Used in FileSource + void closeFileAndStoreMeta(const String & file_name); + + static void assertStreamGood(const std::ifstream & reader); + + template + static V & findInMap(std::unordered_map & map, const K & key) + { + if (auto it = map.find(key); it != map.end()) + return it->second; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} doesn't exist.", key); + } protected: StorageFileLog( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & relative_path_, + const String & path_, + const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, + const String & comment, bool attach); private: std::unique_ptr filelog_settings; - /// user_files_path/ + path_argument/ const String path; + /// For meta file + const String relative_data_path; bool path_is_directory = true; /// If path argument of the table is a regular file, it equals to user_files_path @@ -157,8 +178,10 @@ private: bool updateFileInfos(); - /// Serialize all file meta - void serialize(bool with_end_pos = false) const; + /// Used in shutdown() + void serialize() const; + /// Used in FileSource closeFileAndStoreMeta(file_name); + void serialize(UInt64 inode, const FileMeta & file_meta) const; void deserialize(); }; From 1d0f0993163477ea1d2536c8a14d366411a17a51 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 30 Sep 2021 16:22:26 +0000 Subject: [PATCH 221/950] update test --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02024_storage_filelog_mv.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 2cf23957c9e..d1ed3e5a142 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -18,7 +18,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('a.txt', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/a.txt', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 095c0f05c40..6554f019a9f 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -22,7 +22,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index f73dc988c92..0503552b8dd 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -20,7 +20,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 076e2af44c2..a7293288522 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -22,7 +22,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" From 4390c8f0290069847f1f644df46c825e9253383a Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 30 Sep 2021 17:13:45 +0000 Subject: [PATCH 222/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 18a76b72af8..dca7a678678 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -715,8 +715,14 @@ bool StorageFileLog::updateFileInfos() { String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - auto & file_ctx = findInMap(file_infos.context_by_name, file_name); - file_ctx.status = FileStatus::UPDATED; + /// When new file added and appended, it has two event: DW_ITEM_ADDED + /// and DW_ITEM_MODIFIED, since the order of these two events in the + /// sequence is uncentain, so we may can not find it in file_infos, just + /// skip it, the file info will be handled in DW_ITEM_ADDED case. + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::UPDATED; + } break; } @@ -725,11 +731,12 @@ bool StorageFileLog::updateFileInfos() { String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - auto & file_ctx = findInMap(file_infos.context_by_name, file_name); - file_ctx.status = FileStatus::REMOVED; + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::REMOVED; + } break; } - /// file rename case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { auto file_name = std::filesystem::path(event.path).filename(); @@ -739,9 +746,9 @@ bool StorageFileLog::updateFileInfos() auto inode = getInode(event.path); file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + /// File has been renamed, we should also rename meta file if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { - // rename meta file auto old_name = it->second.file_name; it->second.file_name = file_name; if (std::filesystem::exists(getFullMetaPath(old_name))) From 8c20a184ca4501d4f2713b2b4ce162f23b71a5ac Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 05:06:19 +0000 Subject: [PATCH 223/950] optimize event handle --- .../FileLog/FileLogDirectoryWatcher.cpp | 46 ++++++++----------- .../FileLog/FileLogDirectoryWatcher.h | 5 +- src/Storages/FileLog/StorageFileLog.cpp | 26 +++++------ 3 files changed, 35 insertions(+), 42 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 5bec4fff650..0055d5d3eb1 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -39,53 +39,47 @@ const std::string & FileLogDirectoryWatcher::getPath() const void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemAdded"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemAdded"}; + events.emplace(ev.item.path(), info); } void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemRemoved"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemRemoved"}; + events.emplace(ev.item.path(), info); } - +/// Optimize for MODIFY event, during a streamToViews period, since the log files +/// are append only, there are may a lots of MODIFY events produced for one file. +/// For example, appending 10000 logs into one file will result in 10000 MODIFY event. +/// So, if we record all of these events, it will use a lot of memory, and then we +/// need to handle it one by one in StorageFileLog::updateFileInfos, this is unnecessary +/// because it is equal to just record and handle one MODIY event void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemModified"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + auto event_path = ev.item.path(); + /// Already have MODIFY event for this file + if (auto it = events.find(event_path); it != events.end() && it->second.type == ev.event) + return; + EventInfo info{ev.event, "onItemModified"}; + events.emplace(event_path, info); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemMovedFrom"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemMovedFrom"}; + events.emplace(ev.item.path(), info); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemMovedTo"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemMovedTo"}; + events.emplace(ev.item.path(), info); } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index d21d3d1e1ce..e8ce1a2bbd5 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -12,14 +12,13 @@ class FileLogDirectoryWatcher { public: - struct DirEvent + struct EventInfo { Poco::DirectoryWatcher::DirectoryEventType type; std::string callback; - std::string path; }; - using Events = std::vector; + using Events = std::unordered_map; struct Error { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index dca7a678678..fe5562d72fb 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -691,17 +691,17 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & event : events) + for (const auto & [event_path, event_info] : events) { - switch (event.type) + switch (event_info.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: { - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path)) + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + if (std::filesystem::is_regular_file(event_path)) { - auto file_name = std::filesystem::path(event.path).filename(); - auto inode = getInode(event.path); + auto file_name = std::filesystem::path(event_path).filename(); + auto inode = getInode(event_path); file_infos.file_names.push_back(file_name); @@ -713,8 +713,8 @@ bool StorageFileLog::updateFileInfos() case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { - String file_name = std::filesystem::path(event.path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + String file_name = std::filesystem::path(event_path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); /// When new file added and appended, it has two event: DW_ITEM_ADDED /// and DW_ITEM_MODIFIED, since the order of these two events in the /// sequence is uncentain, so we may can not find it in file_infos, just @@ -729,8 +729,8 @@ bool StorageFileLog::updateFileInfos() case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { - String file_name = std::filesystem::path(event.path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + String file_name = std::filesystem::path(event_path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { it->second.status = FileStatus::REMOVED; @@ -739,11 +739,11 @@ bool StorageFileLog::updateFileInfos() } case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { - auto file_name = std::filesystem::path(event.path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + auto file_name = std::filesystem::path(event_path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); file_infos.file_names.push_back(file_name); - auto inode = getInode(event.path); + auto inode = getInode(event_path); file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); /// File has been renamed, we should also rename meta file From ac9af1ca387d19250f241a971ef96c039687df4c Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 08:24:57 +0000 Subject: [PATCH 224/950] fix wrong offset --- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index e161c69a960..48947bb6304 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -109,11 +109,14 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ Record record; while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) { + /// Need to get offset before reading record from stream + record.offset = file_ctx.reader.tellg(); + record.file_name = file_name; + StorageFileLog::assertStreamGood(file_ctx.reader); std::getline(file_ctx.reader, record.data); - record.file_name = file_name; - record.offset = file_ctx.reader.tellg(); + new_records.emplace_back(record); ++read_records_size; } From b2d13d0b199c7b648718247da310d33774e4f164 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 12 Sep 2021 16:43:22 +0300 Subject: [PATCH 225/950] Add test for expanding macros in RabbitMQ settings. --- .../test_storage_rabbitmq/configs/macros.xml | 8 +++++ .../integration/test_storage_rabbitmq/test.py | 33 ++++++++++++++++++- 2 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_storage_rabbitmq/configs/macros.xml diff --git a/tests/integration/test_storage_rabbitmq/configs/macros.xml b/tests/integration/test_storage_rabbitmq/configs/macros.xml new file mode 100644 index 00000000000..6e9f3390b39 --- /dev/null +++ b/tests/integration/test_storage_rabbitmq/configs/macros.xml @@ -0,0 +1,8 @@ + + + rabbitmq1 + 5672 + macro + JSONEachRow + + diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 9e2752438f8..696294f4bde 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -18,7 +18,7 @@ from . import rabbitmq_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', - main_configs=['configs/rabbitmq.xml'], + main_configs=['configs/rabbitmq.xml', 'configs/macros.xml'], with_rabbitmq=True) @@ -233,6 +233,37 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): rabbitmq_check_result(result, True) +def test_rabbitmq_macros(rabbitmq_cluster): + instance.query(''' + CREATE TABLE test.rabbitmq (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = '{rabbitmq_host}:{rabbitmq_port}', + rabbitmq_exchange_name = '{rabbitmq_exchange_name}', + rabbitmq_format = '{rabbitmq_format}' + ''') + + credentials = pika.PlainCredentials('root', 'clickhouse') + parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) + connection = pika.BlockingConnection(parameters) + channel = connection.channel() + + message = '' + for i in range(50): + message += json.dumps({'key': i, 'value': i}) + '\n' + channel.basic_publish(exchange='macro', routing_key='', body=message) + + connection.close() + time.sleep(1) + + result = '' + while True: + result += instance.query('SELECT * FROM test.rabbitmq ORDER BY key', ignore_error=True) + if rabbitmq_check_result(result): + break + + rabbitmq_check_result(result, True) + + def test_rabbitmq_materialized_view(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.view; From 6eeade23fa4025ad34c62b1bb8805a8255532a1d Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 08:29:19 +0000 Subject: [PATCH 226/950] revert drop table immediately remove unused code fix pvs check fix --- src/Databases/DatabaseAtomic.cpp | 5 ----- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 4 ---- src/Storages/FileLog/StorageFileLog.h | 12 +++++------- src/Storages/IStorage.h | 2 -- 5 files changed, 6 insertions(+), 19 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 26f694cb5dd..5c75f6f1036 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -140,11 +140,6 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na if (table->storesDataOnDisk()) tryRemoveSymlink(table_name); - if (table->dropTableImmediately()) - { - table->drop(); - } - /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c87a9b91640..f273f8a165d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -898,7 +898,7 @@ void DatabaseCatalog::dropTableDataTask() void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) { - if (table.table && !table.table->dropTableImmediately()) + if (table.table) { table.table->drop(); } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 48947bb6304..04af2f829ba 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -10,10 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_READ_FROM_ISTREAM; -} ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 7384543fa36..f181ad570c3 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -49,8 +49,6 @@ public: void drop() override; - bool dropTableImmediately() const override { return true; } - const auto & getFormatName() const { return format_name; } enum class FileStatus @@ -64,15 +62,15 @@ public: struct FileContext { FileStatus status = FileStatus::OPEN; - UInt64 inode; - std::ifstream reader; + UInt64 inode{}; + std::ifstream reader{}; }; struct FileMeta { String file_name; - UInt64 last_writen_position = 0; - UInt64 last_open_end = 0; + UInt64 last_writen_position{}; + UInt64 last_open_end{}; }; using InodeToFileMeta = std::unordered_map; @@ -137,7 +135,7 @@ private: /// If path argument of the table is a regular file, it equals to user_files_path /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path String root_data_path; - /// Database meta_path/ + .table_name/ + /// relative_data_path/ + table_name/ String root_meta_path; FileInfos file_infos; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index f64437e1889..6ce17552ba1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -578,8 +578,6 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } - virtual bool dropTableImmediately() const { return false; } - private: /// Lock required for alter queries (lockForAlter). Always taken for write /// (actually can be replaced with std::mutex, but for consistency we use From ccdcaa21ff1696a1566374117caf49acece8358d Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Fri, 1 Oct 2021 16:13:10 +0300 Subject: [PATCH 227/950] Update install.sh --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index f39bd951f46..764bc891c28 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -48,6 +48,7 @@ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), From 77081f33c29ebc436aec415d220c6a8ffb88e38f Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Fri, 1 Oct 2021 16:13:51 +0300 Subject: [PATCH 228/950] Update install.sh --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 764bc891c28..94ad55504a8 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -48,7 +48,6 @@ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ -ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), @@ -60,6 +59,7 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/executable_pool_dictionary.xml $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/ From d41ef6a9f8b4ae47aadf8c572cd90c04208ad087 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Fri, 1 Oct 2021 17:08:22 +0300 Subject: [PATCH 229/950] Update CORS.xml --- tests/config/config.d/CORS.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/config/config.d/CORS.xml b/tests/config/config.d/CORS.xml index 873821478dc..b96209866a7 100644 --- a/tests/config/config.d/CORS.xml +++ b/tests/config/config.d/CORS.xml @@ -17,5 +17,4 @@ 86400
- - + From 5bb5e04fb3b4c96b3a0e0d2d44dc0359d4958a67 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 14:47:05 +0000 Subject: [PATCH 230/950] fix assert --- src/Storages/FileLog/StorageFileLog.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index fe5562d72fb..ded3016c123 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -83,7 +83,6 @@ StorageFileLog::StorageFileLog( #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif @@ -670,7 +669,6 @@ bool StorageFileLog::updateFileInfos() { #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); assert(file_infos.file_names.size() == 1); #endif @@ -784,7 +782,6 @@ bool StorageFileLog::updateFileInfos() /// These file infos should always have same size(one for one) #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif From 50ef202b12aba727011ea2fe14f588f56e66a2d5 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 1 Oct 2021 18:25:57 +0300 Subject: [PATCH 231/950] Update docs/en/sql-reference/statements/create/view.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 39c5760ecf3..f174d561cc6 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -50,7 +50,7 @@ When creating a materialized view with `TO [db].[table]`, you must not use `POPU A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view. !!! important "Important" - Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views. + Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in the `SELECT` query result, ClickHouse uses a default value, even if the column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views. Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. From bb5c92276d0d9b5d838624b5cc345f33bbb41fdf Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 1 Oct 2021 18:26:09 +0300 Subject: [PATCH 232/950] Update docs/en/sql-reference/statements/create/view.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index f174d561cc6..ec34c57a4cd 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -54,7 +54,7 @@ A materialized view is implemented as follows: when inserting data to the table Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view. -If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it. +If you specify `POPULATE`, the existing table data is inserted into the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it. A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`. From fbe95f9c9d1834e09edd6c04b58ce58d09732f4e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 1 Oct 2021 18:26:17 +0300 Subject: [PATCH 233/950] Update docs/ru/sql-reference/functions/hash-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 98b5ed6df27..975efdae71c 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -263,7 +263,7 @@ SHA512('s') Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов). Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать. -Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при выполнении `SELECT`. +Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`. **Параметры** From 7d5ea307f1d51b434ca44072eaf51aba6e57e992 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 1 Oct 2021 18:26:25 +0300 Subject: [PATCH 234/950] Update docs/ru/sql-reference/functions/hash-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 975efdae71c..18197f88ce3 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -262,7 +262,7 @@ SHA512('s') ``` Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов). -Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать. +Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хеш-функция и вы не можете её выбрать. Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`. **Параметры** From 89f4830180ae0120797d75cc81e46ab5abd2ff2e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 1 Oct 2021 18:26:33 +0300 Subject: [PATCH 235/950] Update docs/ru/sql-reference/statements/create/view.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index 9eb0baf5a98..77bdc7249c7 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -49,7 +49,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na !!! important "Важно" - Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца. + Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse использует значение по умолчанию, даже если столбец не является [Nullable](../../data-types/nullable.md). Безопасной практикой при использовании материализованных представлений считается добавление псевдонимов для каждого столбца. Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление. From 61a7db9612ed67ce4320bda3c193ec07669f4242 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 1 Oct 2021 18:26:55 +0300 Subject: [PATCH 236/950] Update docs/ru/sql-reference/statements/create/view.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/statements/create/view.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index 77bdc7249c7..7ebb154d6b6 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -53,7 +53,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление. -Если указано `POPULATE`, то при создании представления, в него будут вставлены имеющиеся данные таблицы, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Иначе, представление будет содержать только данные, вставляемые в таблицу после создания представления. Не рекомендуется использовать `POPULATE`, так как вставляемые в таблицу данные во время создания представления, не попадут в него. +Если указано `POPULATE`, то при создании представления в него будут добавлены данные, уже содержащиеся в исходной таблице, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Если `POPULATE` не указано, представление будет содержать только данные, добавленные в таблицу после создания представления. Использовать `POPULATE` не рекомендуется, так как в представление не попадут данные, добавляемые в таблицу во время создания представления. Запрос `SELECT` может содержать `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Следует иметь ввиду, что соответствующие преобразования будут выполняться независимо, на каждый блок вставляемых данных. Например, при наличии `GROUP BY`, данные будут агрегироваться при вставке, но только в рамках одной пачки вставляемых данных. Далее, данные не будут доагрегированы. Исключение - использование ENGINE, производящего агрегацию данных самостоятельно, например, `SummingMergeTree`. From a99a6fccc7289fb75bb55a3b47cfda8d144478b0 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sat, 2 Oct 2021 02:34:53 +0800 Subject: [PATCH 237/950] init --- src/Interpreters/getOSKernelVersion.cpp | 37 +++++++++++++++++++++++++ src/Interpreters/getOSKernelVersion.h | 31 +++++++++++++++++++++ 2 files changed, 68 insertions(+) create mode 100644 src/Interpreters/getOSKernelVersion.cpp create mode 100644 src/Interpreters/getOSKernelVersion.h diff --git a/src/Interpreters/getOSKernelVersion.cpp b/src/Interpreters/getOSKernelVersion.cpp new file mode 100644 index 00000000000..44df948be3c --- /dev/null +++ b/src/Interpreters/getOSKernelVersion.cpp @@ -0,0 +1,37 @@ +#if defined(OS_LINUX) +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +String getOSKernelVersion() +{ + struct utsname os_kernel_version; + int buf = uname(&os_kernel_version); + if (buf < 0) + { + throw Exception( + "EFAULT buf is not valid.", + ErrorCodes::BAD_ARGUMENTS); + } + else + { + // std::cout <<"sysname: " << os_kernel_version.sysname << " nodename: " << os_kernel_version.nodename + // << " release: " << os_kernel_version.release << " version: " << os_kernel_version.version + // << " machine: " << os_kernel_version.machine << std::endl; + + return "sysname: " + String(os_kernel_version.sysname) + " nodename: " + String(os_kernel_version.nodename) + + " release: " + String(os_kernel_version.release) + " version: " + String(os_kernel_version.version) + + " machine: " + String(os_kernel_version.machine); + } +} + +} + +#endif \ No newline at end of file diff --git a/src/Interpreters/getOSKernelVersion.h b/src/Interpreters/getOSKernelVersion.h new file mode 100644 index 00000000000..14b42d2a19a --- /dev/null +++ b/src/Interpreters/getOSKernelVersion.h @@ -0,0 +1,31 @@ +#if defined(OS_LINUX) +#pragma once + +#include + +#include +#include + +namespace DB +{ + +/// Returns String with OS Kernel version. +/* To get name and information about current kernel. + For simplicity, the function can be implemented only for Linux. +*/ + +String getOSKernelVersion(); + +// String getSysName(); + +// String getNodeName(); + +// String getReleaseName(); + +// String getVersion(); + +// String getMachineName(); + +} + +#endif \ No newline at end of file From 0bf597374fe239af7da624bbf09c54d9111f9fbf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Oct 2021 05:56:32 +0300 Subject: [PATCH 238/950] More enhancements for query obfuscator --- programs/format/Format.cpp | 5 +++++ src/IO/ReadHelpers.h | 16 ++++++++++++---- src/Parsers/obfuscateQueries.cpp | 11 +++++++++-- 3 files changed, 26 insertions(+), 6 deletions(-) diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index 984543a6c6b..4b0e8ad1ca1 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -25,6 +25,8 @@ #include #include #include +#include +#include #pragma GCC diagnostic ignored "-Wunused-function" @@ -114,6 +116,7 @@ int mainEntryClickHouseFormat(int argc, char ** argv) registerAggregateFunctions(); registerTableFunctions(); registerStorages(); + registerFormats(); std::unordered_set additional_names; @@ -130,6 +133,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv) return FunctionFactory::instance().has(what) || AggregateFunctionFactory::instance().isAggregateFunctionName(what) || TableFunctionFactory::instance().isTableFunctionName(what) + || FormatFactory::instance().isOutputFormat(what) + || FormatFactory::instance().isInputFormat(what) || additional_names.count(what); }; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ca6affbf907..bfb30e8b95c 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -276,29 +276,37 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) { case '+': { - if (has_sign || has_number) + if (has_sign) { + if (has_number) + return ReturnType(true); + if constexpr (throw_exception) throw ParsingException( - "Cannot parse number with multiple sign (+/-) characters or intermediate sign character", + "Cannot parse number with multiple sign (+/-) characters", ErrorCodes::CANNOT_PARSE_NUMBER); else return ReturnType(false); } + has_sign = true; break; } case '-': { - if (has_sign || has_number) + if (has_sign) { + if (has_number) + return ReturnType(true); + if constexpr (throw_exception) throw ParsingException( - "Cannot parse number with multiple sign (+/-) characters or intermediate sign character", + "Cannot parse number with multiple sign (+/-) characters", ErrorCodes::CANNOT_PARSE_NUMBER); else return ReturnType(false); } + if constexpr (is_signed_v) negative = true; else diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index eb0bf5281c9..c0b57d9b1f5 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -38,7 +38,8 @@ const std::unordered_set keywords "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", - "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY" + "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY", "OFFSET", + "TRIM", "LTRIM", "RTRIM", "BOTH", "LEADING", "TRAILING" }; const std::unordered_set keep_words @@ -906,7 +907,13 @@ void obfuscateQueries( /// Write quotes and the obfuscated content inside. result.write(*token.begin); - obfuscateIdentifier({token.begin + 1, token.size() - 2}, result, obfuscate_map, used_nouns, hash_func); + + /// If it is long, just replace it with hash. Long identifiers in queries are usually auto-generated. + if (token.size() > 32) + writeIntText(sipHash64(token.begin + 1, token.size() - 2), result); + else + obfuscateIdentifier({token.begin + 1, token.size() - 2}, result, obfuscate_map, used_nouns, hash_func); + result.write(token.end[-1]); } else if (token.type == TokenType::Number) From 3a56cc5141624e2c7d5555cd8fe18fdebc52e318 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Oct 2021 09:31:53 +0000 Subject: [PATCH 239/950] fix update test --- src/Common/ErrorCodes.cpp | 5 - .../FileLog/FileLogDirectoryWatcher.cpp | 12 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 123 +++++++++--------- .../0_stateless/02023_storage_filelog.sh | 2 +- .../02025_storage_filelog_virtual_col.sh | 2 +- 6 files changed, 70 insertions(+), 76 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9312fde8baa..ee456f81bcf 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -588,12 +588,7 @@ M(618, LZ4_DECODER_FAILED) \ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ - M(621, PATH_DOESNT_EXIST) \ - M(622, FILE_META_NOT_FOUND) \ - M(623, CANNOT_GET_FILE_STAT) \ M(624, NOT_REGULAR_FILE) \ - M(625, READ_META_FILE_FAILED) \ - M(626, FILE_STREAM_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 0055d5d3eb1..6fa6a1605af 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -40,7 +40,7 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemAdded"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } @@ -48,7 +48,7 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemRemoved"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } /// Optimize for MODIFY event, during a streamToViews period, since the log files @@ -62,24 +62,24 @@ void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::Direc std::lock_guard lock(mutex); auto event_path = ev.item.path(); /// Already have MODIFY event for this file - if (auto it = events.find(event_path); it != events.end() && it->second.type == ev.event) + if (auto it = events.find(event_path); it != events.end() && it->second.back().type == ev.event) return; EventInfo info{ev.event, "onItemModified"}; - events.emplace(event_path, info); + events[event_path].emplace_back(info); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedFrom"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedTo"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index e8ce1a2bbd5..ae1853b7a53 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -18,7 +18,7 @@ public: std::string callback; }; - using Events = std::unordered_map; + using Events = std::unordered_map>; struct Error { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ded3016c123..e1218b99332 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -37,10 +37,9 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; - extern const int CANNOT_GET_FILE_STAT; + extern const int CANNOT_STAT; extern const int NOT_REGULAR_FILE; - extern const int READ_META_FILE_FAILED; - extern const int FILE_STREAM_ERROR; + extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; extern const int TABLE_METADATA_ALREADY_EXISTS; } @@ -262,12 +261,12 @@ void StorageFileLog::deserialize() if (!tryReadIntText(inode, in)) { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); } assertChar('\n', in); if (!tryReadIntText(last_written_pos, in)) { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); } meta.file_name = dir_entry.path().filename(); @@ -282,7 +281,7 @@ UInt64 StorageFileLog::getInode(const String & file_name) struct stat file_stat; if (stat(file_name.c_str(), &file_stat)) { - throw Exception(ErrorCodes::CANNOT_GET_FILE_STAT, "Can not get stat info of file {}", file_name); + throw Exception(ErrorCodes::CANNOT_STAT, "Can not get stat info of file {}", file_name); } return file_stat.st_ino; } @@ -377,7 +376,7 @@ void StorageFileLog::assertStreamGood(const std::ifstream & reader) { if (!reader.good()) { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Stream is in bad state."); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Stream is in bad state."); } } @@ -400,7 +399,7 @@ void StorageFileLog::openFilesAndSetPos() auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken.", file); } /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; @@ -689,69 +688,69 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & [event_path, event_info] : events) + for (const auto & [event_path, event_infos] : events) { - switch (event_info.type) + String file_name = std::filesystem::path(event_path).filename(); + for (const auto & event_info : event_infos) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: + switch (event_info.type) { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - if (std::filesystem::is_regular_file(event_path)) + case Poco::DirectoryWatcher::DW_ITEM_ADDED: { - auto file_name = std::filesystem::path(event_path).filename(); - auto inode = getInode(event_path); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + if (std::filesystem::is_regular_file(event_path)) + { + auto inode = getInode(event_path); + + file_infos.file_names.push_back(file_name); + + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + } + break; + } + + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + { + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + /// When new file added and appended, it has two event: DW_ITEM_ADDED + /// and DW_ITEM_MODIFIED, since the order of these two events in the + /// sequence is uncentain, so we may can not find it in file_infos, just + /// skip it, the file info will be handled in DW_ITEM_ADDED case. + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::UPDATED; + } + break; + } + + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + { + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::REMOVED; + } + break; + } + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + { + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); file_infos.file_names.push_back(file_name); - - file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + auto inode = getInode(event_path); file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); - } - break; - } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - { - String file_name = std::filesystem::path(event_path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - /// When new file added and appended, it has two event: DW_ITEM_ADDED - /// and DW_ITEM_MODIFIED, since the order of these two events in the - /// sequence is uncentain, so we may can not find it in file_infos, just - /// skip it, the file info will be handled in DW_ITEM_ADDED case. - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::UPDATED; - } - break; - } - - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - { - String file_name = std::filesystem::path(event_path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::REMOVED; - } - break; - } - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - { - auto file_name = std::filesystem::path(event_path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - - file_infos.file_names.push_back(file_name); - auto inode = getInode(event_path); - file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); - - /// File has been renamed, we should also rename meta file - if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) - { - auto old_name = it->second.file_name; - it->second.file_name = file_name; - if (std::filesystem::exists(getFullMetaPath(old_name))) + /// File has been renamed, we should also rename meta file + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { - std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + auto old_name = it->second.file_name; + it->second.file_name = file_name; + if (std::filesystem::exists(getFullMetaPath(old_name))) + { + std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + } } } } diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 6554f019a9f..b818093dfdb 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -55,7 +55,7 @@ ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" truncate ${user_files_path}/logs/a.txt --size 0 # exception happend -${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index a7293288522..b8b7c980268 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -55,7 +55,7 @@ ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order truncate ${user_files_path}/logs/a.txt --size 0 # exception happend -${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" From 9683513c525ac6c12150d3615758742813953c2e Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Oct 2021 11:03:20 +0000 Subject: [PATCH 240/950] fix build --- src/Storages/FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/FileLogSource.cpp | 2 +- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.h | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index ae1853b7a53..d58a23f402b 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index e2223b3d3c2..40114b4f6f6 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 04af2f829ba..2fc0e0885a1 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index c792978a68a..b4102870c3a 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index e1218b99332..b2b11f22aa9 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f181ad570c3..1ffa04f7c50 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include From ece880184b4b6bfe48a7428cefe26e15953e20f0 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 3 Oct 2021 20:26:48 +0400 Subject: [PATCH 241/950] Reorganiza contrib IDE folders --- cmake/find/capnp.cmake | 2 - cmake/find/cxx.cmake | 2 - cmake/find/unwind.cmake | 1 - contrib/CMakeLists.txt | 93 ++++++++++++++++++++++++++++++++++++----- 4 files changed, 82 insertions(+), 16 deletions(-) diff --git a/cmake/find/capnp.cmake b/cmake/find/capnp.cmake index ee4735bd175..25dfce24ae9 100644 --- a/cmake/find/capnp.cmake +++ b/cmake/find/capnp.cmake @@ -34,8 +34,6 @@ endif() if (CAPNP_LIBRARIES) set (USE_CAPNP 1) elseif(NOT MISSING_INTERNAL_CAPNP_LIBRARY) - add_subdirectory(contrib/capnproto-cmake) - set (CAPNP_LIBRARIES capnpc) set (USE_CAPNP 1) set (USE_INTERNAL_CAPNP_LIBRARY 1) diff --git a/cmake/find/cxx.cmake b/cmake/find/cxx.cmake index b1da125e219..b96ba1e1b65 100644 --- a/cmake/find/cxx.cmake +++ b/cmake/find/cxx.cmake @@ -50,8 +50,6 @@ endif () if (NOT HAVE_LIBCXX AND NOT MISSING_INTERNAL_LIBCXX_LIBRARY) set (LIBCXX_LIBRARY cxx) set (LIBCXXABI_LIBRARY cxxabi) - add_subdirectory(contrib/libcxxabi-cmake) - add_subdirectory(contrib/libcxx-cmake) # Exception handling library is embedded into libcxxabi. diff --git a/cmake/find/unwind.cmake b/cmake/find/unwind.cmake index c9f5f30a5d6..9ae23ae23c7 100644 --- a/cmake/find/unwind.cmake +++ b/cmake/find/unwind.cmake @@ -1,7 +1,6 @@ option (USE_UNWIND "Enable libunwind (better stacktraces)" ${ENABLE_LIBRARIES}) if (USE_UNWIND) - add_subdirectory(contrib/libunwind-cmake) set (UNWIND_LIBRARIES unwind) set (EXCEPTION_HANDLING_LIBRARY ${UNWIND_LIBRARIES}) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 140cc0846ec..2c0ddbc8384 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -1,16 +1,5 @@ # Third-party libraries may have substandard code. -# Put all targets defined here and in added subfolders under "contrib/" folder in GUI-based IDEs by default. -# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they will -# appear not in "contrib/" as originally planned here. -get_filename_component (_current_dir_name "${CMAKE_CURRENT_LIST_DIR}" NAME) -if (CMAKE_FOLDER) - set (CMAKE_FOLDER "${CMAKE_FOLDER}/${_current_dir_name}") -else () - set (CMAKE_FOLDER "${_current_dir_name}") -endif () -unset (_current_dir_name) - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w") @@ -49,6 +38,19 @@ add_subdirectory (replxx-cmake) add_subdirectory (unixodbc-cmake) add_subdirectory (nanodbc-cmake) +if (USE_INTERNAL_LIBCXX_LIBRARY AND NOT MISSING_INTERNAL_LIBCXX_LIBRARY) + add_subdirectory(libcxxabi-cmake) + add_subdirectory(libcxx-cmake) +endif () + +if (USE_UNWIND) + add_subdirectory(libunwind-cmake) +endif () + +if (USE_INTERNAL_CAPNP_LIBRARY AND NOT MISSING_INTERNAL_CAPNP_LIBRARY) + add_subdirectory(capnproto-cmake) +endif () + if (USE_YAML_CPP) add_subdirectory (yaml-cpp-cmake) endif() @@ -347,3 +349,72 @@ endif() if (USE_S2_GEOMETRY) add_subdirectory(s2geometry-cmake) endif() + +# Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. +# Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear +# in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, +# instead of controlling it via CMAKE_FOLDER. + +function (ensure_target_rooted_in _target _folder) + # Read the original FOLDER property value, if any. + get_target_property (_folder_prop "${_target}" FOLDER) + + # Normalize that value, so we avoid possible repetitions in folder names. + + if (NOT _folder_prop) + set (_folder_prop "") + endif () + + if (CMAKE_FOLDER AND _folder_prop MATCHES "^${CMAKE_FOLDER}/(.*)\$") + set (_folder_prop "${CMAKE_MATCH_1}") + endif () + + if (_folder AND _folder_prop MATCHES "^${_folder}/(.*)\$") + set (_folder_prop "${CMAKE_MATCH_1}") + endif () + + if (_folder) + set (_folder_prop "${_folder}/${_folder_prop}") + endif () + + if (CMAKE_FOLDER) + set (_folder_prop "${CMAKE_FOLDER}/${_folder_prop}") + endif () + + message (STATUS "${_target} goes under ${_folder_prop}") + + # Set the updated FOLDER property value back. + set_target_properties ("${_target}" PROPERTIES FOLDER "${_folder_prop}") +endfunction () + +function (ensure_own_targets_are_rooted_in _dir _folder) + get_directory_property (_targets DIRECTORY "${_dir}" BUILDSYSTEM_TARGETS) + foreach (_target IN LISTS _targets) + ensure_target_rooted_in ("${_target}" "${_folder}") + endforeach () +endfunction () + +function (ensure_all_targets_are_rooted_in _dir _folder) + ensure_own_targets_are_rooted_in ("${_dir}" "${_folder}") + + get_property (_sub_dirs DIRECTORY "${_dir}" PROPERTY SUBDIRECTORIES) + foreach (_sub_dir IN LISTS _sub_dirs) + ensure_all_targets_are_rooted_in ("${_sub_dir}" "${_folder}") + endforeach () +endfunction () + +function (organize_ide_folders_2_level _dir) + get_filename_component (_dir_name "${_dir}" NAME) + ensure_own_targets_are_rooted_in ("${_dir}" "${_dir_name}") + + # Note, that we respect only first two levels of nesting, we don't want to + # reorganize target folders further within each third-party dir. + + get_property (_sub_dirs DIRECTORY "${_dir}" PROPERTY SUBDIRECTORIES) + foreach (_sub_dir IN LISTS _sub_dirs) + get_filename_component (_sub_dir_name "${_sub_dir}" NAME) + ensure_all_targets_are_rooted_in ("${_sub_dir}" "${_dir_name}/${_sub_dir_name}") + endforeach () +endfunction () + +organize_ide_folders_2_level ("${CMAKE_CURRENT_LIST_DIR}") From 365a6b469e069a174c9c9924e1eee9795bf72bfd Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 3 Oct 2021 20:30:20 +0400 Subject: [PATCH 242/950] Remove debug message --- contrib/CMakeLists.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2c0ddbc8384..cc5a6dbc9b7 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -381,8 +381,6 @@ function (ensure_target_rooted_in _target _folder) set (_folder_prop "${CMAKE_FOLDER}/${_folder_prop}") endif () - message (STATUS "${_target} goes under ${_folder_prop}") - # Set the updated FOLDER property value back. set_target_properties ("${_target}" PROPERTIES FOLDER "${_folder_prop}") endfunction () From 7be521b024acf1eff98fb3075a376f60a9d667bf Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 3 Oct 2021 21:07:44 +0400 Subject: [PATCH 243/950] Do not manipulate FOLDER property on INTERFACE library targets --- contrib/CMakeLists.txt | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index cc5a6dbc9b7..c671369d126 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -356,6 +356,12 @@ endif() # instead of controlling it via CMAKE_FOLDER. function (ensure_target_rooted_in _target _folder) + # Skip INTERFACE library targets, since FOLDER property is not available for them. + get_target_property (_target_type "${_target}" TYPE) + if (_target_type STREQUAL "INTERFACE_LIBRARY") + return () + endif () + # Read the original FOLDER property value, if any. get_target_property (_folder_prop "${_target}" FOLDER) From 8c82e8b9bc032a45cabe52b04eeaeacf36d71be1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Oct 2021 17:41:08 +0000 Subject: [PATCH 244/950] fix fix --- .../FileLog/FileLogDirectoryWatcher.cpp | 65 ++++++++++++-- src/Storages/FileLog/StorageFileLog.cpp | 89 +++++++++++++------ 2 files changed, 118 insertions(+), 36 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 6fa6a1605af..827d8ec105b 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -39,16 +39,36 @@ const std::string & FileLogDirectoryWatcher::getPath() const void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemAdded"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemRemoved"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } /// Optimize for MODIFY event, during a streamToViews period, since the log files @@ -60,26 +80,55 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + auto event_path = ev.item.path(); - /// Already have MODIFY event for this file - if (auto it = events.find(event_path); it != events.end() && it->second.back().type == ev.event) - return; EventInfo info{ev.event, "onItemModified"}; - events[event_path].emplace_back(info); + /// Already have MODIFY event for this file + if (auto it = events.find(event_path); it != events.end()) + { + if (it->second.back().type == ev.event) + return; + else + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemMovedFrom"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemMovedTo"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index b2b11f22aa9..e0ddd19b1ba 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -81,8 +81,8 @@ StorageFileLog::StorageFileLog( loadFiles(); #ifndef NDEBUG - assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif if (path_is_directory) @@ -200,14 +200,18 @@ void StorageFileLog::loadFiles() { InodeToFileMeta valid_metas; valid_metas.reserve(file_infos.context_by_name.size()); - for (const auto & it : file_infos.meta_by_inode) + for (const auto & [inode, meta] : file_infos.meta_by_inode) { - auto file_name = it.second.file_name; - if (file_infos.context_by_name.contains(file_name)) - valid_metas.emplace(it); + /// Note, here we need to use inode to judge does the meta file is valid. + /// In the case that when a file deleted, then we create new file with the + /// same name, it will have different inode number with stored meta file, + /// so the stored meta file is invalid + if (auto it = file_infos.context_by_name.find(meta.file_name); + it != file_infos.context_by_name.end() && it->second.inode == inode) + valid_metas.emplace(inode, meta); /// Delete meta file from filesystem else - std::filesystem::remove(getFullMetaPath(file_name)); + std::filesystem::remove(getFullMetaPath(meta.file_name)); } file_infos.meta_by_inode.swap(valid_metas); } @@ -215,6 +219,10 @@ void StorageFileLog::loadFiles() void StorageFileLog::serialize() const { + if (!std::filesystem::exists(root_meta_path)) + { + std::filesystem::create_directories(root_meta_path); + } for (const auto & it : file_infos.meta_by_inode) { auto full_name = getFullMetaPath(it.second.file_name); @@ -231,6 +239,10 @@ void StorageFileLog::serialize() const void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const { + if (!std::filesystem::exists(root_meta_path)) + { + std::filesystem::create_directories(root_meta_path); + } auto full_name = getFullMetaPath(file_meta.file_name); if (!std::filesystem::exists(full_name)) { @@ -362,14 +374,21 @@ void StorageFileLog::startup() void StorageFileLog::shutdown() { - if (task) + try { - task->stream_cancelled = true; + if (task) + { + task->stream_cancelled = true; - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + } + closeFilesAndStoreMeta(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } - closeFilesAndStoreMeta(); } void StorageFileLog::assertStreamGood(const std::ifstream & reader) @@ -698,14 +717,22 @@ bool StorageFileLog::updateFileInfos() case Poco::DirectoryWatcher::DW_ITEM_ADDED: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + /// Check if it is a regular file, and new file may be renamed or removed if (std::filesystem::is_regular_file(event_path)) { auto inode = getInode(event_path); file_infos.file_names.push_back(file_name); - file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); - file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) + it->second = FileMeta{.file_name = file_name}; + else + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + it->second = FileContext{.inode = inode}; + else + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); } break; } @@ -718,9 +745,7 @@ bool StorageFileLog::updateFileInfos() /// sequence is uncentain, so we may can not find it in file_infos, just /// skip it, the file info will be handled in DW_ITEM_ADDED case. if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { it->second.status = FileStatus::UPDATED; - } break; } @@ -729,28 +754,36 @@ bool StorageFileLog::updateFileInfos() { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { it->second.status = FileStatus::REMOVED; - } break; } case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - file_infos.file_names.push_back(file_name); - auto inode = getInode(event_path); - file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); - - /// File has been renamed, we should also rename meta file - if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) + /// Similar to DW_ITEM_ADDED, but if it removed from an old file + /// should obtain old meta file and rename meta file + if (std::filesystem::is_regular_file(event_path)) { - auto old_name = it->second.file_name; - it->second.file_name = file_name; - if (std::filesystem::exists(getFullMetaPath(old_name))) + file_infos.file_names.push_back(file_name); + auto inode = getInode(event_path); + + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + it->second = FileContext{.inode = inode}; + else + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + + /// File has been renamed, we should also rename meta file + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { - std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + auto old_name = it->second.file_name; + it->second.file_name = file_name; + if (std::filesystem::exists(getFullMetaPath(old_name))) + std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); } + /// May move from other place, adding new meta info + else + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); } } } From 271f7995c03f5e24a047e692fad06f81181c0d93 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 4 Oct 2021 00:19:37 +0300 Subject: [PATCH 245/950] Fix PR comments. --- .../sql-reference/functions/hash-functions.md | 130 ++--------------- .../sql-reference/functions/hash-functions.md | 132 ++---------------- 2 files changed, 19 insertions(+), 243 deletions(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index e28594540be..20fe6d14e86 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -137,23 +137,29 @@ This is a relatively fast non-cryptographic hash function of average quality for Calculates a 64-bit hash code from any type of integer. It works faster than intHash32. Average quality. -## SHA1 {#sha1} +## SHA1, SHA224, SHA256, SHA512 {#sha} -Calculates SHA-1 hash from a string and returns the resulting set of bytes as [FixedString(20)](../data-types/fixedstring.md). +Calculates SHA-1, SHA-224, SHA-256, SHA-512 hash from a string and returns the resulting set of bytes as [FixedString](../data-types/fixedstring.md). **Syntax** ``` sql SHA1('s') +... +SHA512('s') ``` +The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million). +We recommend using this function only in cases when you need a specific hash function and you can’t select it. +Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in `SELECT` queries. + **Arguments** - `s` — Input string for SHA-1 hash calculation. [String](../data-types/string.md). **Returned value** -- SHA-1 hash as a hex-unencoded FixedString(10). +- SHA hash as a hex-unencoded FixedString. SHA-1 returns as FixedString(20), SHA-224 as FixedString(28), SHA-256 — FixedString(32), SHA-512 — FixedString(64). Type: [FixedString](../data-types/fixedstring.md). @@ -175,124 +181,6 @@ Result: └──────────────────────────────────────────┘ ``` -## SHA224 {#sha224} - -Calculates SHA-224 hash from a string and returns the resulting set of bytes as [FixedString(28)](../data-types/fixedstring.md). - -**Syntax** - -``` sql -SHA224('s') -``` - -**Arguments** - -- `s` — Input string for SHA-224 hash calculation. [String](../data-types/string.md). - -**Returned value** - -- SHA-224 hash as a hex-unencoded FixedString(28). - -Type: [FixedString](../data-types/fixedstring.md). - -**Example** - -Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. - -Query: - -``` sql -SELECT hex(SHA224('abc')); -``` - -Result: - -``` text -┌─hex(SHA224('abc'))───────────────────────────────────────┐ -│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │ -└──────────────────────────────────────────────────────────┘ -``` - -## SHA256 {#sha256} - -Calculates SHA-256 hash from a string and returns the resulting set of bytes as [FixedString(32)](../data-types/fixedstring.md). - -**Syntax** - -``` sql -SHA256('s') -``` - -**Arguments** - -- `s` — Input string for SHA-256 hash calculation. [String](../data-types/string.md). - -**Returned value** - -- SHA-256 hash as a hex-unencoded FixedString(32). - -Type: [FixedString](../data-types/fixedstring.md). - -**Example** - -Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. - -Query: - -``` sql -SELECT hex(SHA256('abc')); -``` - -Result: - -``` text -┌─hex(SHA256('abc'))───────────────────────────────────────────────┐ -│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │ -└──────────────────────────────────────────────────────────────────┘ -``` - -## SHA512 {#sha512} - -Calculates SHA-512 hash from a string and returns the resulting set of bytes as [FixedString(64)](../data-types/fixedstring.md). - -**Syntax** - -``` sql -SHA512('s') -``` - -The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million). -We recommend using this function only in cases when you need a specific hash function and you can’t select it. -Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in `SELECT` queries. - -**Arguments** - -- `s` — Input string for SHA-512 hash calculation. [String](../data-types/string.md). - -**Returned value** - -- SHA-512 hash as a hex-unencoded FixedString(64). - -Type: [FixedString](../data-types/fixedstring.md). - -**Example** - -Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string. - -Query: - -``` sql -SELECT hex(SHA512('abc')); -``` - -Result: - -``` text -┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │ -└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - ## URLHash(url\[, N\]) {#urlhashurl-n} A fast, decent-quality non-cryptographic hash function for a string obtained from a URL using some type of normalization. diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md index 18197f88ce3..f3b14625a8c 100644 --- a/docs/ru/sql-reference/functions/hash-functions.md +++ b/docs/ru/sql-reference/functions/hash-functions.md @@ -137,23 +137,29 @@ SELECT groupBitXor(cityHash64(*)) FROM table Вычисляет 64-битный хэш-код от целого числа любого типа. Работает быстрее, чем intHash32. Качество среднее. -## SHA1 {#sha1} +## SHA1, SHA224, SHA256, SHA512 {#sha} -Вычисляет SHA-1 хеш строки и возвращает полученный набор байт в виде [FixedString(20)](../data-types/fixedstring.md). +Вычисляет SHA-1, SHA-224, SHA-256, SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString](../data-types/fixedstring.md). **Синтаксис** ``` sql SHA1('s') +... +SHA512('s') ``` +Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов). +Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хеш-функция и вы не можете её выбрать. +Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`. + **Параметры** -- `s` — входная строка для вычисления хеша SHA-1. [String](../data-types/string.md). +- `s` — входная строка для вычисления хеша SHA. [String](../data-types/string.md). **Возвращаемое значение** -- Хеш SHA-1 в виде шестнадцатеричной некодированной строки FixedString(20). +- Хеш SHA в виде шестнадцатеричной некодированной строки FixedString. SHA-1 хеш как FixedString(20), SHA-224 как FixedString(28), SHA-256 — FixedString(32), SHA-512 — FixedString(64). Тип: [FixedString](../data-types/fixedstring.md). @@ -175,124 +181,6 @@ SELECT hex(SHA1('abc')); └──────────────────────────────────────────┘ ``` -## SHA224 {#sha224} - -Вычисляет SHA-224 хеш строки и возвращает полученный набор байт в виде [FixedString(28)](../data-types/fixedstring.md). - -**Синтаксис** - -``` sql -SHA224('s') -``` - -**Параметры** - -- `s` — входная строка для вычисления хеша SHA-224. [String](../data-types/string.md). - -**Возвращаемое значение** - -- Хеш SHA-224 в виде шестнадцатеричной некодированной строки FixedString(28). - -Тип: [FixedString](../data-types/fixedstring.md). - -**Пример** - -Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. - -Запрос: - -``` sql -SELECT hex(SHA224('abc')); -``` - -Результат: - -``` text -┌─hex(SHA224('abc'))───────────────────────────────────────┐ -│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │ -└──────────────────────────────────────────────────────────┘ -``` - -## SHA256 {#sha256} - -Вычисляет SHA-256 хеш строки и возвращает полученный набор байт в виде [FixedString(32)](../data-types/fixedstring.md). - -**Синтаксис** - -``` sql -SHA256('s') -``` - -**Параметры** - -- `s` — входная строка для вычисления хеша SHA-256. [String](../data-types/string.md). - -**Возвращаемое значение** - -- Хеш SHA-256 в виде шестнадцатеричной некодированной строки FixedString(32). - -Тип: [FixedString](../data-types/fixedstring.md). - -**Пример** - -Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. - -Запрос: - -``` sql -SELECT hex(SHA256('abc')); -``` - -Результат: - -``` text -┌─hex(SHA256('abc'))───────────────────────────────────────────────┐ -│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │ -└──────────────────────────────────────────────────────────────────┘ -``` - -## SHA512 {#sha512} - -Вычисляет SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString(64)](../data-types/fixedstring.md) - -**Синтаксис** - -``` sql -SHA512('s') -``` - -Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов). -Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хеш-функция и вы не можете её выбрать. -Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`. - -**Параметры** - -- `s` — входная строка для вычисления хеша SHA-512. [String](../data-types/string.md). - -**Возвращаемое значение** - -- Хеш SHA-512 в виде шестнадцатеричной некодированной строки FixedString(64). - -Тип: [FixedString](../data-types/fixedstring.md). - -**Пример** - -Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой. - -Запрос: - -``` sql -SELECT hex(SHA512('abc')); -``` - -Результат: - -``` text -┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │ -└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -``` - ## URLHash(url\[, N\]) {#urlhashurl-n} Быстрая не криптографическая хэш-функция неплохого качества для строки, полученной из URL путём некоторой нормализации. From 57c5d9d3828b69935a71d6f472762bdbce93bb46 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 4 Oct 2021 00:29:46 +0300 Subject: [PATCH 246/950] Update hash-functions.md minor fix --- docs/en/sql-reference/functions/hash-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index d8659b406df..21ed8d33098 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -159,7 +159,7 @@ Even in these cases, we recommend applying the function offline and pre-calculat **Arguments** -- `s` — Input string for SHA-1 hash calculation. [String](../data-types/string.md). +- `s` — Input string for SHA hash calculation. [String](../data-types/string.md). **Returned value** From f40a17c300e023fff1845c94f7c17f7030401c82 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 4 Oct 2021 07:43:28 +0000 Subject: [PATCH 247/950] fix bug in updateFileInfos update --- src/Storages/FileLog/StorageFileLog.cpp | 39 +++++++++++++++++-------- 1 file changed, 27 insertions(+), 12 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index e0ddd19b1ba..ffcd9486f7e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -705,6 +705,12 @@ bool StorageFileLog::updateFileInfos() if (error.has_error) LOG_ERROR(log, "Error happened during watching directory {}: {}", directory_watch->getPath(), error.error_msg); +/// These file infos should always have same size(one for one) before update and after update +#ifndef NDEBUG + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); +#endif + auto events = directory_watch->getEventsAndReset(); for (const auto & [event_path, event_infos] : events) @@ -794,24 +800,33 @@ bool StorageFileLog::updateFileInfos() /// Remove file infos with REMOVE status for (const auto & file_name : file_infos.file_names) { - if (auto it = file_infos.context_by_name.find(file_name); - it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED) + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { - /// Erase meta_by_inode first, otherwise it become invalid - file_infos.meta_by_inode.erase(it->second.inode); - file_infos.context_by_name.erase(it); + if (it->second.status == FileStatus::REMOVED) + { + file_infos.context_by_name.erase(it); - if (std::filesystem::exists(getFullMetaPath(file_name))) - std::filesystem::remove(getFullMetaPath(file_name)); - } - else - { - valid_files.push_back(file_name); + /// We need to check that this ionde does not hold by other file(mv), + /// otherwise, we can not destroy it. + auto inode = it->second.inode; + /// If it's now hold by other file, than the file_name should has + /// been changed during updating file_infos + if (auto meta = file_infos.meta_by_inode.find(inode); + meta != file_infos.meta_by_inode.end() && meta->second.file_name == file_name) + file_infos.meta_by_inode.erase(meta); + + if (std::filesystem::exists(getFullMetaPath(file_name))) + std::filesystem::remove(getFullMetaPath(file_name)); + } + else + { + valid_files.push_back(file_name); + } } } file_infos.file_names.swap(valid_files); - /// These file infos should always have same size(one for one) +/// These file infos should always have same size(one for one) #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); From 45a49b399a268bb9421ae05eca468d5aff640482 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 4 Oct 2021 08:23:02 +0000 Subject: [PATCH 248/950] fix typo --- src/Storages/FileLog/StorageFileLog.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ffcd9486f7e..bf81457995f 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -81,8 +81,8 @@ StorageFileLog::StorageFileLog( loadFiles(); #ifndef NDEBUG - assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif if (path_is_directory) @@ -806,7 +806,7 @@ bool StorageFileLog::updateFileInfos() { file_infos.context_by_name.erase(it); - /// We need to check that this ionde does not hold by other file(mv), + /// We need to check that this inode does not hold by other file(mv), /// otherwise, we can not destroy it. auto inode = it->second.inode; /// If it's now hold by other file, than the file_name should has From 4497f5094e518f0a9d16068c152c88b6bc4c5c98 Mon Sep 17 00:00:00 2001 From: Mikhail <71978106+michon470@users.noreply.github.com> Date: Mon, 4 Oct 2021 16:19:51 +0300 Subject: [PATCH 249/950] Moved changes to this new branch --- .../sql-reference/statements/alter/column.md | 36 ++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 801690afbb2..8f9273c81ba 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -10,7 +10,7 @@ A set of queries that allow changing the table structure. Syntax: ``` sql -ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ... +ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|MODIFY|MATERIALIZE COLUMN ... ``` In the query, specify a list of one or more comma-separated actions. @@ -25,6 +25,7 @@ The following actions are supported: - [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column. - [MODIFY COLUMN](#alter_modify-column) — Changes column’s type, default expression and TTL. - [MODIFY COLUMN REMOVE](#modify-remove) — Removes one of the column properties. +- [MATERIALIZE COLUMN](#materialize-column) — Materializes the column in the parts where the column is missing. These actions are described in detail below. @@ -193,6 +194,39 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; - [REMOVE TTL](ttl.md). +## MATERIALIZE COLUMN {#materialize-column} + +Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier as shown below. + +Syntax: + +```sql +ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; +``` + +**Example:** + +```sql +DROP TABLE IF EXISTS tmp; +SET mutations_sync = 2; +CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple(); +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20; +ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); +SELECT groupArray(x), groupArray(s) FROM tmp; +``` + +**Result:** + +```sql +┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │ +└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +**See Also** + +- [MATERIALIZED](../../statements/create/table.md#materialized). + ## Limitations {#alter-query-limitations} The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot. From 52c5f2da7203eaaae8ea819bc8ef405dafacb1c2 Mon Sep 17 00:00:00 2001 From: Mikhail <71978106+michon470@users.noreply.github.com> Date: Mon, 4 Oct 2021 16:41:50 +0300 Subject: [PATCH 250/950] =?UTF-8?q?=D0=9F=D0=B5=D1=80=D0=B5=D0=B2=D0=BE?= =?UTF-8?q?=D0=B4=20+=20=D0=B4=D0=BE=D0=BF=D0=BE=D0=BB=D0=BD=D0=B5=D0=BD?= =?UTF-8?q?=D0=B8=D0=B5=20=D0=B0=D0=BD=D0=B3=D0=BB=20=D0=B2=D0=B5=D1=80?= =?UTF-8?q?=D1=81=D0=B8=D0=B8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql-reference/statements/alter/column.md | 2 +- .../sql-reference/statements/alter/column.md | 38 +++++++++++++++++-- 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 8f9273c81ba..31874ef208d 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -204,7 +204,7 @@ Syntax: ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; ``` -**Example:** +**Example with the creation of new column:** ```sql DROP TABLE IF EXISTS tmp; diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 9f59c79bfdd..366caf6a2a0 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -10,7 +10,7 @@ toc_title: "Манипуляции со столбцами" Синтаксис: ``` sql -ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ... +ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|MODIFY|MATERIALIZE COLUMN ... ``` В запросе можно указать сразу несколько действий над одной таблицей через запятую. @@ -20,11 +20,12 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN - [ADD COLUMN](#alter_add-column) — добавляет столбец в таблицу; - [DROP COLUMN](#alter_drop-column) — удаляет столбец; -- [RENAME COLUMN](#alter_rename-column) — переименовывает существующий столбец. +- [RENAME COLUMN](#alter_rename-column) — переименовывает существующий столбец; - [CLEAR COLUMN](#alter_clear-column) — сбрасывает все значения в столбце для заданной партиции; - [COMMENT COLUMN](#alter_comment-column) — добавляет комментарий к столбцу; -- [MODIFY COLUMN](#alter_modify-column) — изменяет тип столбца, выражение для значения по умолчанию и TTL. -- [MODIFY COLUMN REMOVE](#modify-remove) — удаляет какое-либо из свойств столбца. +- [MODIFY COLUMN](#alter_modify-column) — изменяет тип столбца, выражение для значения по умолчанию и TTL; +- [MODIFY COLUMN REMOVE](#modify-remove) — удаляет какое-либо из свойств столбца; +- [MATERIALIZE COLUMN](#materialize-column) — делает столбец материализованным (MATERIALIZED) в его частях, у которых отсутствуют значения. Подробное описание для каждого действия приведено ниже. @@ -193,6 +194,35 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; - [REMOVE TTL](ttl.md). +## MATERIALIZE COLUMN {#materialize-column} + +С помощью этого запроса можно сделать столбец таблицы материализованным (`MATERIALIZED`) в его частях, у которых отсутствуют значения. Это полезно, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`). Если вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо большим, для него может оказаться целесообразным использовать `MATERIALIZE COLUMN`. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`, как показано ниже. + +Синтаксис: + +```sql +ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; +``` + +**Пример использования при создании нового столбца:** + +```sql +DROP TABLE IF EXISTS tmp; +SET mutations_sync = 2; +CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple(); +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20; +ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); +SELECT groupArray(x), groupArray(s) FROM tmp; +``` + +**Результат:** + +```sql +┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │ +└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + ## Ограничения запроса ALTER {#ogranicheniia-zaprosa-alter} Запрос `ALTER` позволяет создавать и удалять отдельные элементы (столбцы) вложенных структур данных, но не вложенные структуры данных целиком. Для добавления вложенной структуры данных, вы можете добавить столбцы с именем вида `name.nested_name` и типом `Array(T)` - вложенная структура данных полностью эквивалентна нескольким столбцам-массивам с именем, имеющим одинаковый префикс до точки. From 873805d4da99cc4e7db467f86ddc94336436d68a Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 4 Oct 2021 12:33:05 +0000 Subject: [PATCH 251/950] update poco update fix unbundled build --- contrib/poco | 2 +- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 2 +- src/Storages/FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.h | 3 ++- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/contrib/poco b/contrib/poco index 46c80daf1b0..5966f3a72aa 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 46c80daf1b015aa10474ce82e3d24b578c6ae422 +Subproject commit 5966f3a72aa54f904433a0ffea3f0d7655c2fac2 diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 827d8ec105b..3b4f89d50f5 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -3,7 +3,7 @@ #include FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_shared(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) + : path(path_), dw(std::make_unique(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { /// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported on Linux. /// On other platforms, a file rename or move operation will be reported via a diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index d58a23f402b..158c629b5fd 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -45,7 +45,7 @@ protected: private: const std::string path; - std::shared_ptr dw; + std::unique_ptr dw; Poco::Logger * log; diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 1ffa04f7c50..929be2690e1 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -12,9 +12,10 @@ #include #include -#include #include +#include #include +#include namespace DB { From 200f655a2f76b3ac23ba10ff54a09da83931af37 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 25 Aug 2021 09:50:30 +0300 Subject: [PATCH 252/950] Fix writing marks in StorageLog. --- src/Storages/StorageLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..691e431907a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -214,8 +214,7 @@ public: , storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(std::move(lock_)) - , marks_stream( - storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) + , marks_stream(storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Append)) { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); From bb32432943c1e04f442d6df83dcb31faa0c269c8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 5 Oct 2021 00:13:18 +0300 Subject: [PATCH 253/950] more fixes --- .../MergeTree/MutateFromLogEntryTask.cpp | 6 ++- .../MergeTree/MutateFromLogEntryTask.h | 1 + .../MergeTree/MutatePlainMergeTreeTask.cpp | 6 ++- .../MergeTree/MutatePlainMergeTreeTask.h | 1 + src/Storages/StorageMergeTree.cpp | 49 ++----------------- src/Storages/StorageMergeTree.h | 1 - .../01162_strange_mutations.reference | 24 +++++++++ .../0_stateless/01162_strange_mutations.sh | 17 +++++-- 8 files changed, 55 insertions(+), 50 deletions(-) diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 7cca7012a2c..a6c70e1db4f 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -75,9 +75,13 @@ std::pair MutateFromLogEntry stopwatch_ptr = std::make_unique(); + fake_query_context = Context::createCopy(storage.getContext()); + fake_query_context->makeQueryContext(); + fake_query_context->setCurrentQueryId(""); + mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_mutated_part, metadata_snapshot, commands, merge_mutate_entry.get(), - entry.create_time, storage.getContext(), reserved_space, table_lock_holder); + entry.create_time, fake_query_context, reserved_space, table_lock_holder); return {true, [this] (const ExecutionStatus & execution_status) { diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.h b/src/Storages/MergeTree/MutateFromLogEntryTask.h index 2fa4f0c0388..9aaad14dd4c 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.h +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.h @@ -42,6 +42,7 @@ private: MergeTreeData::MutableDataPartPtr new_part{nullptr}; FutureMergedMutatedPartPtr future_mutated_part{nullptr}; + ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; }; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 71128b31cdb..b7768531373 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -43,9 +43,13 @@ void MutatePlainMergeTreeTask::prepare() merge_list_entry.get()); }; + fake_query_context = Context::createCopy(storage.getContext()); + fake_query_context->makeQueryContext(); + fake_query_context->setCurrentQueryId(""); + mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_part, metadata_snapshot, merge_mutate_entry->commands, merge_list_entry.get(), - time(nullptr), storage.getContext(), merge_mutate_entry->tagger->reserved_space, table_lock_holder); + time(nullptr), fake_query_context, merge_mutate_entry->tagger->reserved_space, table_lock_holder); } bool MutatePlainMergeTreeTask::executeStep() diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index e8f210f4175..7e332ad3334 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -74,6 +74,7 @@ private: IExecutableTask::TaskResultCallback task_result_callback; + ContextMutablePtr fake_query_context; MutateTaskPtr mutate_task; }; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index faa16ac875b..3829b3813aa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -927,12 +927,16 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( { try { + auto fake_query_context = Context::createCopy(getContext()); + fake_query_context->makeQueryContext(); + fake_query_context->setCurrentQueryId(""); MutationsInterpreter interpreter( - shared_from_this(), metadata_snapshot, commands_for_size_validation, getContext(), false); + shared_from_this(), metadata_snapshot, commands_for_size_validation, fake_query_context, false); commands_size += interpreter.evaluateCommandsSize(); } catch (...) { + tryLogCurrentException(log); MergeTreeMutationEntry & entry = it->second; entry.latest_fail_time = time(nullptr); entry.latest_fail_reason = getCurrentExceptionMessage(false); @@ -965,49 +969,6 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( return {}; } -bool StorageMergeTree::mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & merge_mutate_entry, TableLockHolder & table_lock_holder) -{ - auto & future_part = merge_mutate_entry.future_part; - - auto merge_list_entry = getContext()->getMergeList().insert(getStorageID(), future_part); - Stopwatch stopwatch; - MutableDataPartPtr new_part; - - auto write_part_log = [&] (const ExecutionStatus & execution_status) - { - writePartLog( - PartLogElement::MUTATE_PART, - execution_status, - stopwatch.elapsed(), - future_part->name, - new_part, - future_part->parts, - merge_list_entry.get()); - }; - - try - { - auto task = merger_mutator.mutatePartToTemporaryPart( - future_part, metadata_snapshot, merge_mutate_entry.commands, merge_list_entry.get(), - time(nullptr), getContext(), merge_mutate_entry.tagger->reserved_space, table_lock_holder); - - new_part = executeHere(task); - - renameTempPartAndReplace(new_part); - - updateMutationEntriesErrors(future_part, true, ""); - write_part_log({}); - } - catch (...) - { - updateMutationEntriesErrors(future_part, false, getCurrentExceptionMessage(false)); - write_part_log(ExecutionStatus::fromCurrentException()); - throw; - } - - return true; -} - bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assignee) //-V657 { if (shutdown_called) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 08cce514d71..11379359844 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -178,7 +178,6 @@ private: std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); - bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); Int64 getCurrentMutationVersion( const DataPartPtr & part, diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference index e09dccd4c4a..4c925f849d6 100644 --- a/tests/queries/0_stateless/01162_strange_mutations.reference +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -1,8 +1,32 @@ +MergeTree 1 2 2 0 +50 6225 0 +0 +50 6225 1900 +ReplicatedMergeTree 1 2 2 0 +50 6225 0 +2 +50 6225 0 +Memory +1 +2 +2 +0 +50 6225 0 +0 +50 6225 1900 +Join +1 +2 +2 +0 +50 6225 0 +0 +50 6225 0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index c4166a88e42..c75bee47e4e 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -4,18 +4,29 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -declare -a engines=("MergeTree" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1')") +declare -a engines=("MergeTree order by n" "ReplicatedMergeTree('/test/01162/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') order by n" "Memory" "Join(ALL, FULL, n)") + +$CLICKHOUSE_CLIENT -q "CREATE OR REPLACE VIEW t1 AS SELECT number * 10 AS id, number * 100 AS value FROM numbers(20)" for engine in "${engines[@]}" do $CLICKHOUSE_CLIENT -q "drop table if exists t" - $CLICKHOUSE_CLIENT -q "create table t (n int) engine=$engine order by n" + $CLICKHOUSE_CLIENT -q "create table t (n int) engine=$engine" + $CLICKHOUSE_CLIENT -q "select engine from system.tables where database=currentDatabase() and name='t'" $CLICKHOUSE_CLIENT -q "insert into t values (1)" $CLICKHOUSE_CLIENT -q "insert into t values (2)" $CLICKHOUSE_CLIENT -q "select * from t order by n" $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select * from (select * from t where n global in (1::Int32)))" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 full join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" $CLICKHOUSE_CLIENT -q "select count() from t" $CLICKHOUSE_CLIENT -q "drop table t" + + $CLICKHOUSE_CLIENT -q "drop table if exists test" + $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" + $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" + # FIXME it's not clear if the following query should fail or not + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -c "Unknown function" + $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" + $CLICKHOUSE_CLIENT -q "drop table test" done From a638c40fdcfd1b8b01153713e070664c1c38976f Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Tue, 5 Oct 2021 09:08:16 +0800 Subject: [PATCH 254/950] add getOSKernelVersion --- src/Interpreters/getOSKernelVersion.cpp | 18 ++++++------------ src/Interpreters/getOSKernelVersion.h | 12 +----------- 2 files changed, 7 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/getOSKernelVersion.cpp b/src/Interpreters/getOSKernelVersion.cpp index 44df948be3c..c4b4564f46e 100644 --- a/src/Interpreters/getOSKernelVersion.cpp +++ b/src/Interpreters/getOSKernelVersion.cpp @@ -7,28 +7,22 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; + extern const int SYSTEM_ERROR; } String getOSKernelVersion() { - struct utsname os_kernel_version; - int buf = uname(&os_kernel_version); + struct utsname os_kernel_info; + int buf = uname(&os_kernel_info); if (buf < 0) { throw Exception( - "EFAULT buf is not valid.", - ErrorCodes::BAD_ARGUMENTS); + "EFAULT buffer is not valid.", + ErrorCodes::SYSTEM_ERROR); } else { - // std::cout <<"sysname: " << os_kernel_version.sysname << " nodename: " << os_kernel_version.nodename - // << " release: " << os_kernel_version.release << " version: " << os_kernel_version.version - // << " machine: " << os_kernel_version.machine << std::endl; - - return "sysname: " + String(os_kernel_version.sysname) + " nodename: " + String(os_kernel_version.nodename) - + " release: " + String(os_kernel_version.release) + " version: " + String(os_kernel_version.version) - + " machine: " + String(os_kernel_version.machine); + return String(os_kernel_info.sysname) + " " + String(os_kernel_info.release); } } diff --git a/src/Interpreters/getOSKernelVersion.h b/src/Interpreters/getOSKernelVersion.h index 14b42d2a19a..fc3c7583aef 100644 --- a/src/Interpreters/getOSKernelVersion.h +++ b/src/Interpreters/getOSKernelVersion.h @@ -1,5 +1,5 @@ -#if defined(OS_LINUX) #pragma once +#if defined(OS_LINUX) #include @@ -16,16 +16,6 @@ namespace DB String getOSKernelVersion(); -// String getSysName(); - -// String getNodeName(); - -// String getReleaseName(); - -// String getVersion(); - -// String getMachineName(); - } #endif \ No newline at end of file From 1ea23230845d5a043d850f892a2d3bd22bed5082 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 5 Oct 2021 06:40:00 +0000 Subject: [PATCH 255/950] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 5966f3a72aa..39fd359765a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 5966f3a72aa54f904433a0ffea3f0d7655c2fac2 +Subproject commit 39fd359765a3a77b46d94ec3c5def3c7802a920f From 9ff8a2391d310807807f391275945950274e3d14 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 5 Oct 2021 03:39:15 +0300 Subject: [PATCH 256/950] Add test. --- ...02047_log_family_data_file_sizes.reference | 47 ++++++++++++++++++ .../02047_log_family_data_file_sizes.sh | 48 +++++++++++++++++++ 2 files changed, 95 insertions(+) create mode 100644 tests/queries/0_stateless/02047_log_family_data_file_sizes.reference create mode 100755 tests/queries/0_stateless/02047_log_family_data_file_sizes.sh diff --git a/tests/queries/0_stateless/02047_log_family_data_file_sizes.reference b/tests/queries/0_stateless/02047_log_family_data_file_sizes.reference new file mode 100644 index 00000000000..b802026a0dd --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_data_file_sizes.reference @@ -0,0 +1,47 @@ +Log: +empty: +1 element: +1 a +__marks.mrk +sizes.json +x.bin +y.bin +3 elements: +1 a +22 bc +333 def +__marks.mrk greater size +sizes.json +x.bin greater size +y.bin greater size + +TinyLog: +empty: +1 element: +1 a +sizes.json +x.bin +y.bin +3 elements: +1 a +22 bc +333 def +sizes.json +x.bin greater size +y.bin greater size + +StripeLog: +empty: +1 element: +1 a +data.bin +index.mrk +sizes.json +3 elements: +1 a +22 bc +333 def +data.bin greater size +index.mrk greater size +sizes.json + diff --git a/tests/queries/0_stateless/02047_log_family_data_file_sizes.sh b/tests/queries/0_stateless/02047_log_family_data_file_sizes.sh new file mode 100755 index 00000000000..e7c1cb5d71e --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_data_file_sizes.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +declare -a engines=("Log" "TinyLog" "StripeLog") +for engine in "${engines[@]}" +do + echo "$engine:" + + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS tbl" + $CLICKHOUSE_CLIENT --query="CREATE TABLE tbl(x UInt32, y String) ENGINE=$engine" + data_dir=$($CLICKHOUSE_CLIENT --query="SELECT data_paths[1] FROM system.tables WHERE name='tbl' AND database=currentDatabase()") + + echo "empty:" + find "$data_dir"* 2>/dev/null + + echo "1 element:" + $CLICKHOUSE_CLIENT --query="INSERT INTO tbl VALUES (1, 'a')" + $CLICKHOUSE_CLIENT --query="SELECT * FROM tbl ORDER BY x" + declare -A file_sizes + for name in $(find "$data_dir"* -print0 | xargs -0 -n 1 basename | sort); do + file_path=$data_dir$name + file_size=$(stat -c%s "$file_path") + file_sizes[$name]=$file_size + echo $name + done + + echo "3 elements:" + $CLICKHOUSE_CLIENT --query="INSERT INTO tbl VALUES (22, 'bc'), (333, 'def')" + $CLICKHOUSE_CLIENT --query="SELECT * FROM tbl ORDER BY x" + for name in $(find "$data_dir"* -print0 | xargs -0 -n 1 basename | sort); do + file_path=$data_dir$name + file_size=$(stat -c%s "$file_path") + old_file_size=${file_sizes[$name]} + if [ "$name" == "sizes.json" ]; then + cmp="" + elif (( file_size > old_file_size )); then + cmp="greater size" + else + cmp="unexpected size ($file_size, old_size=$old_file_size)" + fi + echo $name $cmp + done + + echo +done From 82e6ac8fa2cdba5ef016bbe4278d7c17888dafb7 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Tue, 5 Oct 2021 13:39:18 +0300 Subject: [PATCH 257/950] Update HTTPHandler.cpp --- src/Server/HTTPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 413cfe18696..1036d5031f7 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -24,7 +24,7 @@ #include #include #include -#include "common/logger_useful.h" +#include #include #include #include From 0709b2e690df038fc00a671991734081cc606527 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 5 Oct 2021 13:28:45 +0000 Subject: [PATCH 258/950] fix --- src/Storages/FileLog/FileLogSource.cpp | 7 ++++++- src/Storages/FileLog/FileLogSource.h | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 40114b4f6f6..de46c880301 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -38,7 +38,7 @@ FileLogSource::FileLogSource( buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } -FileLogSource::~FileLogSource() +void FileLogSource::onFinish() { auto & file_infos = storage.getFileInfos(); @@ -56,7 +56,12 @@ FileLogSource::~FileLogSource() Chunk FileLogSource::generate() { if (!buffer || buffer->noRecords()) + { + /// There is no onFinish for ISource, we call it + /// when no records return to close files + onFinish(); return {}; + } MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index d257bb54f06..f57b1096749 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -28,7 +28,7 @@ public: bool noRecords() { return !buffer || buffer->noRecords(); } - virtual ~FileLogSource() override; + void onFinish(); protected: Chunk generate() override; From 9b1a39fdb9c4b79d2f045f88d97a1ce33c7d4797 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Tue, 5 Oct 2021 17:43:33 +0300 Subject: [PATCH 259/950] refactor after move common to base --- src/Server/HTTPHandler.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 1036d5031f7..9edef8a7223 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -24,15 +24,15 @@ #include #include #include -#include +#include #include #include #include #include #include -#include -#include +#include +#include #include #if !defined(ARCADIA_BUILD) From f2a62ef9eec023bf54c450ff078f7ee7bef08f3a Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 5 Oct 2021 17:38:18 +0000 Subject: [PATCH 260/950] try to fix data race found in DirectoryWatcher --- src/Storages/FileLog/FileLogDirectoryWatcher.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 158c629b5fd..310b5bde388 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -45,13 +45,19 @@ protected: private: const std::string path; + + /// Note, in order to avoid data race found by fuzzer, put events before dw, + /// such that when this class desctruction, dw will be destructed before events. + /// The data race is because dw create a seperate thread to monitor file events + /// and put into events, then if we destruct events first, the monitor thread still + /// running, it may access events during events destruction, leads to data race. + Events events; + std::unique_ptr dw; Poco::Logger * log; std::mutex mutex; - Events events; - Error error; }; From fbfdd605eea214466bbd8d32a58f214aa5e5ca8e Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 5 Oct 2021 21:58:49 +0300 Subject: [PATCH 261/950] Update metrica.md Update ru with for hits_100m_obfuscated --- docs/ru/getting-started/example-datasets/metrica.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index c82048a445e..27105ca8488 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -38,6 +38,9 @@ $ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads= $ # теперь создадим таблицу $ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" $ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" +# for hits_100m_obfuscated +clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" + $ # импортируем данные $ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 $ # опционально можно оптимизировать таблицу From af94e30a955dbfe271f412ee5ebe384994448f8e Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 5 Oct 2021 22:12:44 +0300 Subject: [PATCH 262/950] Update H3 functions Update en and add ru draft. --- docs/en/sql-reference/functions/geo/h3.md | 14 +-- docs/ru/sql-reference/functions/geo/h3.md | 112 +++++++++++++++++++++- 2 files changed, 118 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 3c3ed7b8932..9cdd3bcf947 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -481,7 +481,7 @@ Type: [UInt64](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT h3ToParent(599405990164561919, 3) as parent; +SELECT h3ToParent(599405990164561919, 3) AS parent; ``` Result: @@ -515,7 +515,7 @@ Type: [String](../../../sql-reference/data-types/string.md). Query: ``` sql -SELECT h3ToString(617420388352917503) as h3_string; +SELECT h3ToString(617420388352917503) AS h3_string; ``` Result: @@ -549,7 +549,7 @@ stringToH3(index_str) Query: ``` sql -SELECT stringToH3('89184926cc3ffff') as index; +SELECT stringToH3('89184926cc3ffff') AS index; ``` Result: @@ -583,7 +583,7 @@ h3GetResolution(index) Query: ``` sql -SELECT h3GetResolution(617420388352917503) as res; +SELECT h3GetResolution(617420388352917503) AS res; ``` Result: @@ -620,7 +620,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT h3IsResClassIII(617420388352917503) as res; +SELECT h3IsResClassIII(617420388352917503) AS res; ``` Result: @@ -657,7 +657,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT SELECT h3IsPentagon(644721767722457330) as pentagon; +SELECT h3IsPentagon(644721767722457330) AS pentagon; ``` Result: @@ -693,7 +693,7 @@ Type: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql- Query: ``` sql -SELECT SELECT h3GetFaces(599686042433355775) as faces; +SELECT h3GetFaces(599686042433355775) AS faces; ``` Result: diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md index bc47ca72a39..e8871d856c4 100644 --- a/docs/ru/sql-reference/functions/geo/h3.md +++ b/docs/ru/sql-reference/functions/geo/h3.md @@ -548,7 +548,7 @@ h3GetResolution(index) Запрос: ``` sql -SELECT h3GetResolution(617420388352917503) as res; +SELECT h3GetResolution(617420388352917503) AS res; ``` Результат: @@ -559,3 +559,113 @@ SELECT h3GetResolution(617420388352917503) as res; └─────┘ ``` +## h3IsResClassIII {#h3isresclassIII} + +Returns whether [H3](#h3index) index has a resolution with Class III orientation. + +**Синтаксис** + +``` sql +h3IsResClassIII(index) +``` + +**Параметр** + +- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- `1` — Index has a resolution with Class III orientation. +- `0` — Index doesn't have a resolution with Class III orientation. + +Тип: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3IsResClassIII(617420388352917503) AS res; +``` + +Результат: + +``` text +┌─res─┐ +│ 1 │ +└─────┘ +``` + +## h3IsPentagon {#h3ispentagon } + +Returns whether this [H3](#h3index) index represents a pentagonal cell. + +**Синтаксис** + +``` sql +h3IsPentagon(index) +``` + +**Параметр** + +- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- `1` — Index represents a pentagonal cell. +- `0` — Index doesn't represent a pentagonal cell. + +Тип: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3IsPentagon(644721767722457330) AS pentagon; +``` + +Результат: + +``` text +┌─pentagon─┐ +│ 0 │ +└──────────┘ +``` + +## h3GetFaces {#h3getfaces} + +Returns icosahedron faces intersected by a given [H3](#h3index) index. + +**Синтаксис** + +``` sql +h3GetFaces(index) +``` + +**Параметр** + +- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Array containing icosahedron faces intersected by a given H3 index. + +Тип: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT h3GetFaces(599686042433355775) AS faces; +``` + +Результат: + +``` text +┌─faces─┐ +│ [7] │ +└───────┘ + +[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3) From acdb8233c3426b78d3c0fc075fd2da7b1b723d92 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 6 Oct 2021 08:08:49 +0000 Subject: [PATCH 263/950] fix fix fix --- .../FileLog/FileLogDirectoryWatcher.h | 4 +- src/Storages/FileLog/FileLogSource.cpp | 26 ++--- src/Storages/FileLog/FileLogSource.h | 11 +++ src/Storages/FileLog/StorageFileLog.cpp | 96 ++++++++++++++++--- src/Storages/FileLog/StorageFileLog.h | 9 +- 5 files changed, 120 insertions(+), 26 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 310b5bde388..00ee7ddfc40 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -47,8 +47,8 @@ private: const std::string path; /// Note, in order to avoid data race found by fuzzer, put events before dw, - /// such that when this class desctruction, dw will be destructed before events. - /// The data race is because dw create a seperate thread to monitor file events + /// such that when this class destruction, dw will be destructed before events. + /// The data race is because dw create a separate thread to monitor file events /// and put into events, then if we destruct events first, the monitor thread still /// running, it may access events during events destruction, leads to data race. Events events; diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index de46c880301..eb8cb2b28f9 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -36,21 +36,17 @@ FileLogSource::FileLogSource( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); + + const auto & file_infos = storage.getFileInfos(); + + size_t files_per_stream = file_infos.file_names.size() / max_streams_number; + start = stream_number * files_per_stream; + end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; } void FileLogSource::onFinish() { - auto & file_infos = storage.getFileInfos(); - - size_t files_per_stream = file_infos.file_names.size() / max_streams_number; - size_t start = stream_number * files_per_stream; - size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; - - /// Each stream responsible for close it's files and store meta - for (size_t i = start; i < end; ++i) - { - storage.closeFileAndStoreMeta(file_infos.file_names[i]); - } + storage.closeFilesAndStoreMeta(start, end); } Chunk FileLogSource::generate() @@ -60,6 +56,7 @@ Chunk FileLogSource::generate() /// There is no onFinish for ISource, we call it /// when no records return to close files onFinish(); + finished = true; return {}; } @@ -105,7 +102,11 @@ Chunk FileLogSource::generate() } if (total_rows == 0) + { + onFinish(); + finished = true; return {}; + } auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); @@ -121,6 +122,9 @@ Chunk FileLogSource::generate() auto converting_actions = std::make_shared(std::move(converting_dag)); converting_actions->execute(result_block); + /// After generate each block, store metas into disk + storage.storeMetas(start, end); + return Chunk(result_block.getColumns(), result_block.rows()); } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index f57b1096749..6f94ef2470a 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -30,6 +30,12 @@ public: void onFinish(); + virtual ~FileLogSource() override + { + if (!finished) + onFinish(); + } + protected: Chunk generate() override; @@ -49,6 +55,11 @@ private: Block non_virtual_header; Block virtual_header; + + /// The start pos and end pos of files responsible by this stream, + /// does not includind end + int start; + int end; }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index bf81457995f..431d88b9f7b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -223,17 +223,26 @@ void StorageFileLog::serialize() const { std::filesystem::create_directories(root_meta_path); } - for (const auto & it : file_infos.meta_by_inode) + for (const auto & [inode, meta] : file_infos.meta_by_inode) { - auto full_name = getFullMetaPath(it.second.file_name); + auto full_name = getFullMetaPath(meta.file_name); if (!std::filesystem::exists(full_name)) { FS::createFile(full_name); } + else + { + auto last_pos = getLastWrittenPos(meta.file_name); + if (last_pos > meta.last_writen_position) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", + full_name); + } WriteBufferFromFile out(full_name); - writeIntText(it.first, out); + writeIntText(inode, out); writeChar('\n', out); - writeIntText(it.second.last_writen_position, out); + writeIntText(meta.last_writen_position, out); } } @@ -248,6 +257,15 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const { FS::createFile(full_name); } + else + { + auto last_pos = getLastWrittenPos(file_meta.file_name); + if (last_pos > file_meta.last_writen_position) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", + full_name); + } WriteBufferFromFile out(full_name); writeIntText(inode, out); writeChar('\n', out); @@ -288,6 +306,23 @@ void StorageFileLog::deserialize() } } +UInt64 StorageFileLog::getLastWrittenPos(const String & file_name) const +{ + ReadBufferFromFile in(getFullMetaPath(file_name)); + UInt64 _, last_written_pos; + + if (!tryReadIntText(_, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); + } + assertChar('\n', in); + if (!tryReadIntText(last_written_pos, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); + } + return last_written_pos; +} + UInt64 StorageFileLog::getInode(const String & file_name) { struct stat file_stat; @@ -307,7 +342,19 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - /// We need this lock, in case read and streamToViews execute at the same time + auto table_id = getStorageID(); + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + /// If there are MVs depended on this table, we just forbid reading + if (dependencies_count) + { + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Can not read from table {}, because it has been depended by other tables.", + table_id.getTableName()); + } + + /// We need this lock, in case read and streamToViews execute at the same time. + /// In case of MV attached during reading std::lock_guard lock(status_mutex); updateFileInfos(); @@ -440,14 +487,41 @@ void StorageFileLog::closeFilesAndStoreMeta() serialize(); } -void StorageFileLog::closeFileAndStoreMeta(const String & file_name) +void StorageFileLog::closeFilesAndStoreMeta(int start, int end) { - auto & file_ctx = findInMap(file_infos.context_by_name, file_name); - if (file_ctx.reader.is_open()) - file_ctx.reader.close(); +#ifndef NDEBUG + assert(start >= 0); + assert(start < end); + assert(end <= file_infos.file_names.size()); +#endif - auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); - serialize(file_ctx.inode, meta); + for (int i = start; i < end; ++i) + { + auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); + + if (file_ctx.reader.is_open()) + file_ctx.reader.close(); + + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + serialize(file_ctx.inode, meta); + } +} + +void StorageFileLog::storeMetas(int start, int end) +{ +#ifndef NDEBUG + assert(start >= 0); + assert(start < end); + assert(end <= file_infos.file_names.size()); +#endif + + for (int i = start; i < end; ++i) + { + auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); + + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + serialize(file_ctx.inode, meta); + } } size_t StorageFileLog::getMaxBlockSize() const diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 929be2690e1..0251544f33f 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -97,10 +97,14 @@ public: static UInt64 getInode(const String & file_name); void openFilesAndSetPos(); + /// Used in shutdown() void closeFilesAndStoreMeta(); - /// Used in FileSource - void closeFileAndStoreMeta(const String & file_name); + /// Used in FileLogSource when finish generating all blocks + void closeFilesAndStoreMeta(int start, int end); + + /// Used in FileLogSource after generating every block + void storeMetas(int start, int end); static void assertStreamGood(const std::ifstream & reader); @@ -183,6 +187,7 @@ private: void serialize(UInt64 inode, const FileMeta & file_meta) const; void deserialize(); + UInt64 getLastWrittenPos(const String & file_name) const; }; } From c801fa65dfb319c7b2e91df975e592b1ab0a1c08 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 6 Oct 2021 10:37:58 +0000 Subject: [PATCH 264/950] fix and add new test fix fix --- src/Databases/DatabaseAtomic.cpp | 3 + src/Storages/FileLog/FileLogSource.h | 4 +- src/Storages/FileLog/StorageFileLog.cpp | 58 ++++++++----------- src/Storages/FileLog/StorageFileLog.h | 12 +++- src/Storages/IStorage.h | 3 + .../02026_storage_filelog_largefile.reference | 3 + .../02026_storage_filelog_largefile.sh | 45 ++++++++++++++ 7 files changed, 90 insertions(+), 38 deletions(-) create mode 100644 tests/queries/0_stateless/02026_storage_filelog_largefile.reference create mode 100755 tests/queries/0_stateless/02026_storage_filelog_largefile.sh diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 5c75f6f1036..ae90f1a6900 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -140,6 +140,9 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na if (table->storesDataOnDisk()) tryRemoveSymlink(table_name); + if (table->dropTableImmediately()) + table->drop(); + /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 6f94ef2470a..0ff71c1af87 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -58,8 +58,8 @@ private: /// The start pos and end pos of files responsible by this stream, /// does not includind end - int start; - int end; + size_t start; + size_t end; }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 431d88b9f7b..112e68acd08 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -232,12 +232,7 @@ void StorageFileLog::serialize() const } else { - auto last_pos = getLastWrittenPos(meta.file_name); - if (last_pos > meta.last_writen_position) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", - full_name); + checkOffsetIsValid(full_name, meta.last_writen_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); @@ -259,12 +254,7 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const } else { - auto last_pos = getLastWrittenPos(file_meta.file_name); - if (last_pos > file_meta.last_writen_position) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", - full_name); + checkOffsetIsValid(full_name, file_meta.last_writen_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); @@ -306,23 +296,6 @@ void StorageFileLog::deserialize() } } -UInt64 StorageFileLog::getLastWrittenPos(const String & file_name) const -{ - ReadBufferFromFile in(getFullMetaPath(file_name)); - UInt64 _, last_written_pos; - - if (!tryReadIntText(_, in)) - { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); - } - assertChar('\n', in); - if (!tryReadIntText(last_written_pos, in)) - { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); - } - return last_written_pos; -} - UInt64 StorageFileLog::getInode(const String & file_name) { struct stat file_stat; @@ -487,7 +460,7 @@ void StorageFileLog::closeFilesAndStoreMeta() serialize(); } -void StorageFileLog::closeFilesAndStoreMeta(int start, int end) +void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { #ifndef NDEBUG assert(start >= 0); @@ -495,7 +468,7 @@ void StorageFileLog::closeFilesAndStoreMeta(int start, int end) assert(end <= file_infos.file_names.size()); #endif - for (int i = start; i < end; ++i) + for (size_t i = start; i < end; ++i) { auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); @@ -507,7 +480,7 @@ void StorageFileLog::closeFilesAndStoreMeta(int start, int end) } } -void StorageFileLog::storeMetas(int start, int end) +void StorageFileLog::storeMetas(size_t start, size_t end) { #ifndef NDEBUG assert(start >= 0); @@ -515,7 +488,7 @@ void StorageFileLog::storeMetas(int start, int end) assert(end <= file_infos.file_names.size()); #endif - for (int i = start; i < end; ++i) + for (size_t i = start; i < end; ++i) { auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); @@ -524,6 +497,25 @@ void StorageFileLog::storeMetas(int start, int end) } } +void StorageFileLog::checkOffsetIsValid(const String & full_name, UInt64 offset) +{ + ReadBufferFromFile in(full_name); + UInt64 _, last_written_pos; + + if (!tryReadIntText(_, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + } + assertChar('\n', in); + if (!tryReadIntText(last_written_pos, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + } + if (last_written_pos > offset) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Last stored last_written_pos in meta file {} is bigger than current last_written_pos.", full_name); +} + size_t StorageFileLog::getMaxBlockSize() const { return filelog_settings->max_block_size.changed ? filelog_settings->max_block_size.value diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 0251544f33f..f33f6320e1a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -50,6 +50,12 @@ public: void drop() override; + /// We need to call drop() immediately to remove meta data directory, + /// otherwise, if another filelog table with same name created before + /// the table be dropped finally, then its meta data directory will + /// be deleted by this table drop finally + bool dropTableImmediately() override { return true; } + const auto & getFormatName() const { return format_name; } enum class FileStatus @@ -101,10 +107,10 @@ public: /// Used in shutdown() void closeFilesAndStoreMeta(); /// Used in FileLogSource when finish generating all blocks - void closeFilesAndStoreMeta(int start, int end); + void closeFilesAndStoreMeta(size_t start, size_t end); /// Used in FileLogSource after generating every block - void storeMetas(int start, int end); + void storeMetas(size_t start, size_t end); static void assertStreamGood(const std::ifstream & reader); @@ -187,7 +193,7 @@ private: void serialize(UInt64 inode, const FileMeta & file_meta) const; void deserialize(); - UInt64 getLastWrittenPos(const String & file_name) const; + static void checkOffsetIsValid(const String & full_name, UInt64 offset); }; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6ce17552ba1..38f24e0ce29 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -578,6 +578,9 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } + /// Do we should call table->drop immediately when drop table + virtual bool dropTableImmediately() { return false; } + private: /// Lock required for alter queries (lockForAlter). Always taken for write /// (actually can be replaced with std::mutex, but for consistency we use diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.reference b/tests/queries/0_stateless/02026_storage_filelog_largefile.reference new file mode 100644 index 00000000000..95240890a95 --- /dev/null +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.reference @@ -0,0 +1,3 @@ +2000000 +2000000 +2000000 diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh new file mode 100755 index 00000000000..8ee6f2e6a0c --- /dev/null +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Data preparation. +# Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: +# "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +mkdir -p ${user_files_path}/logs/ + +rm -rf ${user_files_path}/logs/* + +for i in {1..200} +do + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt32, v UInt32) engine=FileLog('${user_files_path}/logs/', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select count() from file_log " + +for i in {201..400} +do + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" +done + +${CLICKHOUSE_CLIENT} --query "select count() from file_log " + +for i in {401..600} +do + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" +done + +${CLICKHOUSE_CLIENT} --query "select count() from file_log " + +${CLICKHOUSE_CLIENT} --query "drop table file_log;" + +rm -rf ${user_files_path}/logs From 1550c167bb8b725376968d2b1f2779c669f59a3a Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 6 Oct 2021 18:14:51 +0300 Subject: [PATCH 265/950] Update ru translation. --- docs/en/sql-reference/functions/geo/h3.md | 2 +- docs/ru/sql-reference/functions/geo/h3.md | 26 +++++++++++------------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 9cdd3bcf947..410cb9d3cc2 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -631,7 +631,7 @@ Result: └─────┘ ``` -## h3IsPentagon {#h3ispentagon } +## h3IsPentagon {#h3ispentagon} Returns whether this [H3](#h3index) index represents a pentagonal cell. diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md index e8871d856c4..cd807ade04a 100644 --- a/docs/ru/sql-reference/functions/geo/h3.md +++ b/docs/ru/sql-reference/functions/geo/h3.md @@ -6,7 +6,7 @@ toc_title: "Функции для работы с индексами H3" [H3](https://eng.uber.com/h3/) — это система геокодирования, которая делит поверхность Земли на равные шестигранные ячейки. Система поддерживает иерархию (вложенность) ячеек, т.е. каждый "родительский" шестигранник может быть поделен на семь одинаковых вложенных "дочерних" шестигранников, и так далее. -Уровень вложенности назвается `разрешением` и может принимать значение от `0` до `15`, где `0` соответствует `базовым` ячейкам самого верхнего уровня (наиболее крупным). +Уровень вложенности называется `разрешением` и может принимать значение от `0` до `15`, где `0` соответствует `базовым` ячейкам самого верхнего уровня (наиболее крупным). Для каждой точки, имеющей широту и долготу, можно получить 64-битный индекс H3, соответствующий номеру шестигранной ячейки, где эта точка находится. @@ -561,7 +561,7 @@ SELECT h3GetResolution(617420388352917503) AS res; ## h3IsResClassIII {#h3isresclassIII} -Returns whether [H3](#h3index) index has a resolution with Class III orientation. +Проверяет, имеет ли индекс [H3](#h3index) разрешение с ориентацией Class III. **Синтаксис** @@ -571,12 +571,12 @@ h3IsResClassIII(index) **Параметр** -- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `index` — порядковый номер шестигранника. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). **Возвращаемые значения** -- `1` — Index has a resolution with Class III orientation. -- `0` — Index doesn't have a resolution with Class III orientation. +- `1` — индекс имеет разрешение с ориентацией Class III. +- `0` — индекс не имеет разрешения с ориентацией Class III. Тип: [UInt8](../../../sql-reference/data-types/int-uint.md). @@ -596,9 +596,9 @@ SELECT h3IsResClassIII(617420388352917503) AS res; └─────┘ ``` -## h3IsPentagon {#h3ispentagon } +## h3IsPentagon {#h3ispentagon} -Returns whether this [H3](#h3index) index represents a pentagonal cell. +Проверяет, является ли указанный индекс [H3](#h3index) пятиугольной ячейкой. **Синтаксис** @@ -608,12 +608,12 @@ h3IsPentagon(index) **Параметр** -- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `index` — порядковый номер шестигранника. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). **Возвращаемые значения** -- `1` — Index represents a pentagonal cell. -- `0` — Index doesn't represent a pentagonal cell. +- `1` — индекс представляет собой пятиугольную ячейку. +- `0` — индекс не является пятиугольной ячейкой. Тип: [UInt8](../../../sql-reference/data-types/int-uint.md). @@ -635,7 +635,7 @@ SELECT h3IsPentagon(644721767722457330) AS pentagon; ## h3GetFaces {#h3getfaces} -Returns icosahedron faces intersected by a given [H3](#h3index) index. +Возвращает все грани многоугольника (икосаэдра), пересекаемые заданным [H3](#h3index) индексом. **Синтаксис** @@ -645,11 +645,11 @@ h3GetFaces(index) **Параметр** -- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). **Возвращаемое значение** -- Array containing icosahedron faces intersected by a given H3 index. +- Массив, содержащий грани многоугольника (икосаэдра), пересекаемые заданным H3 индексом. Тип: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)). From 259da1ccf07e8ed788ed7a418884c91801bff1fa Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 6 Oct 2021 18:32:55 +0300 Subject: [PATCH 266/950] Update h3.md --- docs/ru/sql-reference/functions/geo/h3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md index cd807ade04a..6bc6943ec93 100644 --- a/docs/ru/sql-reference/functions/geo/h3.md +++ b/docs/ru/sql-reference/functions/geo/h3.md @@ -668,4 +668,4 @@ SELECT h3GetFaces(599686042433355775) AS faces; │ [7] │ └───────┘ -[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3) +[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3/) From 4894588f2751189a55b0dce9ca218e4b0040ec7b Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 6 Oct 2021 19:50:05 +0000 Subject: [PATCH 267/950] description improved new example --- .../functions/other-functions.md | 39 ++++++++++++++++--- 1 file changed, 33 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 44702f4097f..2bb38684eb3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2357,8 +2357,8 @@ Result: ## shardNum {#shard-num} -Returns the number of a shard which executes the query for a distributed query. -If query is not distributed then *constant value* is returned. +Returns the index of a shard which processes a part of data for a distributed query. Indices are started from `1`. +If a query is not distributed then constant value `0` is returned. **Syntax** @@ -2368,14 +2368,39 @@ shardNum() **Returned value** -- Shard number. +- Shard index or constant `0`. Type: [UInt32](../../sql-reference/data-types/int-uint.md). +**Example** + +In the following example a configuration with two shards is used. The query is executed on the [system.one](../../operations/system-tables/one.md) table on every shard. + +Query: + +``` sql +CREATE TABLE shard_num_example (dummy UInt8) + ENGINE=Distributed(test_cluster_two_shards_localhost, system, one, dummy); +SELECT dummy, shardNum(), shardCount() FROM shard_num_example; +``` + +Result: + +``` text +┌─dummy─┬─shardNum()─┬─shardCount()─┐ +│ 0 │ 2 │ 2 │ +│ 0 │ 1 │ 2 │ +└───────┴────────────┴──────────────┘ +``` + +**See Also** + +- [Distributed Table Engine](../../engines/table-engines/special/distributed.md) + ## shardCount {#shard-count} -Returns the total number of shards which execute a distributed query. -If query is not distributed then *constant value* is returned. +Returns the total number of shards for a distributed query. +If a query is not distributed then constant value `0` is returned. **Syntax** @@ -2385,8 +2410,10 @@ shardCount() **Returned value** -- Total number of shards. +- Total number of shards or `0`. Type: [UInt32](../../sql-reference/data-types/int-uint.md). +**See Also** +- [shardNum()](#shard-num) function example also contains `shardCount()` function call. From c2533b974394c2f1b4c356fa8408ad9dc0526d0f Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 6 Oct 2021 20:13:46 +0000 Subject: [PATCH 268/950] constant or column note added for other functions --- docs/en/sql-reference/functions/date-time-functions.md | 1 + docs/en/sql-reference/functions/other-functions.md | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f54ef635e0c..b85f105758b 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -26,6 +26,7 @@ SELECT ## timeZone {#timezone} Returns the timezone of the server. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 2bb38684eb3..afcc9563b58 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -8,6 +8,7 @@ toc_title: Other ## hostName() {#hostname} Returns a string with the name of the host that this function was performed on. For distributed processing, this is the name of the remote server host, if the function is performed on a remote server. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## getMacro {#getmacro} @@ -691,10 +692,12 @@ Returns the largest value of a and b. ## uptime() {#uptime} Returns the server’s uptime in seconds. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## version() {#version} Returns the version of the server as a string. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## blockNumber {#blocknumber} @@ -2101,6 +2104,7 @@ UNSUPPORTED_METHOD ## tcpPort {#tcpPort} Returns [native interface](../../interfaces/tcp.md) TCP port number listened by this server. +If it is executed in the context of a distributed table, then it generates a normal column, otherwise it produces a constant value. **Syntax** From 95154305177f77d245d33493258e28e5df443fbc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Oct 2021 00:17:14 +0300 Subject: [PATCH 269/950] Fix compilation with glibc 2.34 (MINSIGSTKSZ defined as sysconf(_SC_SIGSTKSZ)) In glibc 2.34 MINSIGSTKSZ had been defined to sysconf(_SC_SIGSTKSZ) [1]. [1]: https://sourceware.org/git/?p=glibc.git;a=commit;h=6c57d320484988e87e446e2e60ce42816bf51d53 --- src/Common/ThreadStatus.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index b1d76c4660e..c0190cac58a 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -44,7 +44,7 @@ namespace struct ThreadStack { ThreadStack() - : data(aligned_alloc(getPageSize(), size)) + : data(aligned_alloc(getPageSize(), getSize())) { /// Add a guard page /// (and since the stack grows downward, we need to protect the first page). @@ -56,12 +56,11 @@ struct ThreadStack free(data); } - static size_t getSize() { return size; } + static size_t getSize() { return std::max(16 << 10, MINSIGSTKSZ); } void * getData() const { return data; } private: /// 16 KiB - not too big but enough to handle error. - static constexpr size_t size = std::max(16 << 10, MINSIGSTKSZ); void * data; }; From 28f8e1c297738c79446d52f835192de52cd9fd14 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 6 Oct 2021 15:48:23 +0000 Subject: [PATCH 270/950] fix update test --- src/Storages/FileLog/StorageFileLog.cpp | 15 ++++----------- src/Storages/FileLog/StorageFileLog.h | 5 ++--- .../0_stateless/02022_storage_filelog_one_file.sh | 2 +- .../queries/0_stateless/02023_storage_filelog.sh | 2 +- .../02025_storage_filelog_virtual_col.sh | 2 +- .../02026_storage_filelog_largefile.sh | 2 ++ 6 files changed, 11 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 112e68acd08..6489c9270ca 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -403,7 +403,10 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } - closeFilesAndStoreMeta(); + /// If no reading call and threadFunc, the log files will never + /// be opened, also just leave the work of close files and + /// store meta to streams. because if we close files in here, + /// may result in data race with unfinishing reading pipeline } catch (...) { @@ -450,16 +453,6 @@ void StorageFileLog::openFilesAndSetPos() serialize(); } -void StorageFileLog::closeFilesAndStoreMeta() -{ - for (auto & it : file_infos.context_by_name) - { - if (it.second.reader.is_open()) - it.second.reader.close(); - } - serialize(); -} - void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { #ifndef NDEBUG diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f33f6320e1a..82c7a2dc9f9 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -104,9 +104,8 @@ public: void openFilesAndSetPos(); - /// Used in shutdown() - void closeFilesAndStoreMeta(); - /// Used in FileLogSource when finish generating all blocks + /// Used in FileLogSource when finish generating all blocks. + /// Each stream responsible for close its files and store meta. void closeFilesAndStoreMeta(size_t start, size_t end); /// Used in FileLogSource after generating every block diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index d1ed3e5a142..600e537a352 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index b818093dfdb..b695c270835 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index b8b7c980268..781253c2184 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh index 8ee6f2e6a0c..a6c5d19287e 100755 --- a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -16,6 +16,8 @@ mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* +chmod 777 ${user_files_path}/logs/ + for i in {1..200} do ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" From d9959a4f22bec685f084a5be160ff8de1470d786 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 05:56:09 +0000 Subject: [PATCH 271/950] fix --- src/Storages/FileLog/FileLogSource.cpp | 2 ++ src/Storages/FileLog/FileLogSource.h | 1 + src/Storages/FileLog/StorageFileLog.cpp | 16 ++++++++++++++++ src/Storages/FileLog/StorageFileLog.h | 9 +++++++++ 4 files changed, 28 insertions(+) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index eb8cb2b28f9..0c266178142 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -42,6 +42,8 @@ FileLogSource::FileLogSource( size_t files_per_stream = file_infos.file_names.size() / max_streams_number; start = stream_number * files_per_stream; end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; + + storage.increaseStreams(); } void FileLogSource::onFinish() diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 0ff71c1af87..cdf60cd4fc1 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -34,6 +34,7 @@ public: { if (!finished) onFinish(); + storage.reduceStreams(); } protected: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 6489c9270ca..fb13137ce31 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; extern const int TABLE_METADATA_ALREADY_EXISTS; + extern const int CANNOT_SELECT; } namespace @@ -326,6 +327,11 @@ Pipe StorageFileLog::read( table_id.getTableName()); } + if (running_streams.load(std::memory_order_relaxed)) + { + throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); + } + /// We need this lock, in case read and streamToViews execute at the same time. /// In case of MV attached during reading std::lock_guard lock(status_mutex); @@ -364,6 +370,16 @@ Pipe StorageFileLog::read( return Pipe::unitePipes(std::move(pipes)); } +void StorageFileLog::increaseStreams() +{ + running_streams.fetch_add(1, std::memory_order_relaxed); +} + +void StorageFileLog::reduceStreams() +{ + running_streams.fetch_sub(1, std::memory_order_relaxed); +} + void StorageFileLog::drop() { try diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 82c7a2dc9f9..bdab85866c0 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -122,6 +122,9 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} doesn't exist.", key); } + void increaseStreams(); + void reduceStreams(); + protected: StorageFileLog( const StorageID & table_id_, @@ -169,6 +172,12 @@ private: }; std::shared_ptr task; + /// In order to avoid data race, using a naive trick to forbid execute two select + /// simultaneously, although read is not useful in this engine. Using an atomic + /// variable to records current unfinishing streams, then if have unfinishing streams, + /// later select should forbid to execute. + std::atomic running_streams = 0; + using TaskThread = BackgroundSchedulePool::TaskHolder; void loadFiles(); From 127a0f35910a097b5240d68cc60f5e30b9e697ac Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 07:31:41 +0000 Subject: [PATCH 272/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index fb13137ce31..63337b9b5c2 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -879,8 +879,6 @@ bool StorageFileLog::updateFileInfos() { if (it->second.status == FileStatus::REMOVED) { - file_infos.context_by_name.erase(it); - /// We need to check that this inode does not hold by other file(mv), /// otherwise, we can not destroy it. auto inode = it->second.inode; @@ -892,6 +890,7 @@ bool StorageFileLog::updateFileInfos() if (std::filesystem::exists(getFullMetaPath(file_name))) std::filesystem::remove(getFullMetaPath(file_name)); + file_infos.context_by_name.erase(it); } else { From 07113cb6f63781eda49ab25475c59a5c89093d86 Mon Sep 17 00:00:00 2001 From: Federico Ceratto Date: Thu, 7 Oct 2021 12:50:56 +0100 Subject: [PATCH 273/950] Link FAQ from Debian installation --- docs/en/getting-started/install.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 508cd51e9f8..06186842809 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -27,10 +27,11 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun {% include 'install/deb.sh' %} ``` -If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). +You can replace `stable` with `lts` or `testing` to use different [“release trains”](../faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/). + #### Packages {#packages} - `clickhouse-common-static` — Installs ClickHouse compiled binary files. From c41923c5958067f487b31a27f860cc1e775accdc Mon Sep 17 00:00:00 2001 From: Haavard Kvaalen Date: Thu, 7 Oct 2021 13:29:38 +0200 Subject: [PATCH 274/950] MaterializedMySQL: Update GTID set at end of transaction We would update the set of seen GTIDs as soon as we saw a GTID_EVENT, which arrives before a transaction. This would mostly work fine, but if we lost the connection to MySQL in the middle of a large transaction we would persist that the transaction had been processed as soon as the transaction had started. When the connection was reestablished, we would not process the transaction again, which meant that we only applied parts of it. Fix this by updating the seen GTIDs at the end of the transaction instead. --- src/Core/MySQL/MySQLReplication.cpp | 29 ++++++++++++++++-- src/Core/MySQL/MySQLReplication.h | 4 +++ .../materialize_with_ddl.py | 30 +++++++++++++++++++ .../test_materialized_mysql_database/test.py | 5 ++++ 4 files changed, 65 insertions(+), 3 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 9c90b2ff220..b5468d15edc 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -105,12 +105,16 @@ namespace MySQLReplication if (query.starts_with("BEGIN") || query.starts_with("COMMIT")) { typ = QUERY_EVENT_MULTI_TXN_FLAG; + if (!query.starts_with("COMMIT")) + transaction_complete = false; } else if (query.starts_with("XA")) { if (query.starts_with("XA ROLLBACK")) throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR); typ = QUERY_EVENT_XA; + if (!query.starts_with("XA COMMIT")) + transaction_complete = false; } else if (query.starts_with("SAVEPOINT")) { @@ -711,9 +715,26 @@ namespace MySQLReplication { switch (event->header.type) { - case FORMAT_DESCRIPTION_EVENT: - case QUERY_EVENT: + case FORMAT_DESCRIPTION_EVENT: { + binlog_pos = event->header.log_pos; + break; + } + case QUERY_EVENT: { + auto query = std::static_pointer_cast(event); + if (query->transaction_complete && pending_gtid) + { + gtid_sets.update(*pending_gtid); + pending_gtid.reset(); + } + binlog_pos = event->header.log_pos; + break; + } case XID_EVENT: { + if (pending_gtid) + { + gtid_sets.update(*pending_gtid); + pending_gtid.reset(); + } binlog_pos = event->header.log_pos; break; } @@ -724,9 +745,11 @@ namespace MySQLReplication break; } case GTID_EVENT: { + if (pending_gtid) + gtid_sets.update(*pending_gtid); auto gtid_event = std::static_pointer_cast(event); binlog_pos = event->header.log_pos; - gtid_sets.update(gtid_event->gtid); + pending_gtid = gtid_event->gtid; break; } default: diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index a57cc246eaa..cb67ce73de9 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -383,6 +383,7 @@ namespace MySQLReplication String schema; String query; QueryType typ = QUERY_EVENT_DDL; + bool transaction_complete = true; QueryEvent(EventHeader && header_) : EventBase(std::move(header_)), thread_id(0), exec_time(0), schema_len(0), error_code(0), status_len(0) @@ -536,6 +537,9 @@ namespace MySQLReplication void update(BinlogEventPtr event); void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_); void dump(WriteBuffer & out) const; + + private: + std::optional pending_gtid; }; class IFlavor : public MySQLProtocol::IMySQLReadPacket diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 23fa9894a84..5f6daea24ac 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -980,3 +980,33 @@ def mysql_settings_test(clickhouse_node, mysql_node, service_name): clickhouse_node.query("DROP DATABASE test_database") mysql_node.query("DROP DATABASE test_database") +def materialized_mysql_large_transaction(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS largetransaction") + clickhouse_node.query("DROP DATABASE IF EXISTS largetransaction") + mysql_node.query("CREATE DATABASE largetransaction") + + mysql_node.query("CREATE TABLE largetransaction.test_table (" + "`key` INT NOT NULL PRIMARY KEY AUTO_INCREMENT, " + "`value` INT NOT NULL) ENGINE = InnoDB;") + num_rows = 200000 + rows_per_insert = 5000 + values = ",".join(["(1)" for _ in range(rows_per_insert)]) + for i in range(num_rows//rows_per_insert): + mysql_node.query(f"INSERT INTO largetransaction.test_table (`value`) VALUES {values};") + + + clickhouse_node.query("CREATE DATABASE largetransaction ENGINE = MaterializedMySQL('{}:3306', 'largetransaction', 'root', 'clickhouse')".format(service_name)) + check_query(clickhouse_node, "SELECT COUNT() FROM largetransaction.test_table", f"{num_rows}\n") + + mysql_node.query("UPDATE largetransaction.test_table SET value = 2;") + + # Attempt to restart clickhouse after it has started processing + # the transaction, but before it has completed it. + while int(clickhouse_node.query("SELECT COUNT() FROM largetransaction.test_table WHERE value = 2")) == 0: + time.sleep(0.2) + clickhouse_node.restart_clickhouse() + + check_query(clickhouse_node, "SELECT COUNT() FROM largetransaction.test_table WHERE value = 2", f"{num_rows}\n") + + clickhouse_node.query("DROP DATABASE largetransaction") + mysql_node.query("DROP DATABASE largetransaction") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 18cb5b3b87c..feade1b60a0 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -237,3 +237,8 @@ def test_materialize_with_enum(started_cluster, started_mysql_8_0, started_mysql def test_mysql_settings(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_8_0, "mysql80") + +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) +def test_large_transaction(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): + materialize_with_ddl.materialized_mysql_large_transaction(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.materialized_mysql_large_transaction(clickhouse_node, started_mysql_5_7, "mysql57") From bd1ea5cd1630c2b8d73c8234c840cdf84649a903 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 12:15:36 +0000 Subject: [PATCH 275/950] try fix undefine in ifstream --- .../FileLog/ReadBufferFromFileLog.cpp | 20 +++++++++++----- src/Storages/FileLog/StorageFileLog.cpp | 24 +++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 3 ++- 3 files changed, 30 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 2fc0e0885a1..cf52b2a2ccc 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -10,6 +10,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; +} ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, @@ -102,23 +106,27 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); + if (!file_ctx.reader) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized.", file_meta.file_name); + + auto & reader = file_ctx.reader.value(); Record record; - while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) + while (read_records_size < need_records_size && static_cast(reader.tellg()) < file_meta.last_open_end) { /// Need to get offset before reading record from stream - record.offset = file_ctx.reader.tellg(); + record.offset = reader.tellg(); record.file_name = file_name; - StorageFileLog::assertStreamGood(file_ctx.reader); + StorageFileLog::assertStreamGood(reader); - std::getline(file_ctx.reader, record.data); + std::getline(reader, record.data); new_records.emplace_back(record); ++read_records_size; } - UInt64 current_position = file_ctx.reader.tellg(); - StorageFileLog::assertStreamGood(file_ctx.reader); + UInt64 current_position = reader.tellg(); + StorageFileLog::assertStreamGood(reader); file_meta.last_writen_position = current_position; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 63337b9b5c2..c9b29a5a9d1 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -445,14 +445,15 @@ void StorageFileLog::openFilesAndSetPos() auto & file_ctx = findInMap(file_infos.context_by_name, file); if (file_ctx.status != FileStatus::NO_CHANGE) { - file_ctx.reader.open(getFullDataPath(file)); - assertStreamGood(file_ctx.reader); + file_ctx.reader.emplace(getFullDataPath(file)); + auto & reader = file_ctx.reader.value(); + assertStreamGood(reader); - file_ctx.reader.seekg(0, file_ctx.reader.end); - assertStreamGood(file_ctx.reader); + reader.seekg(0, reader.end); + assertStreamGood(reader); - auto file_end = file_ctx.reader.tellg(); - assertStreamGood(file_ctx.reader); + auto file_end = reader.tellg(); + assertStreamGood(reader); auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) @@ -462,8 +463,8 @@ void StorageFileLog::openFilesAndSetPos() /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; - file_ctx.reader.seekg(meta.last_writen_position); - assertStreamGood(file_ctx.reader); + reader.seekg(meta.last_writen_position); + assertStreamGood(reader); } } serialize(); @@ -481,8 +482,11 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); - if (file_ctx.reader.is_open()) - file_ctx.reader.close(); + if (file_ctx.reader) + { + if ((*file_ctx.reader).is_open()) + (*file_ctx.reader).close(); + } auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); serialize(file_ctx.inode, meta); diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index bdab85866c0..f3b85d47313 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -70,7 +71,7 @@ public: { FileStatus status = FileStatus::OPEN; UInt64 inode{}; - std::ifstream reader{}; + std::optional reader = std::nullopt; }; struct FileMeta From afd69ef8336356c4e38505c8881dec58e9b6169a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 7 Oct 2021 15:04:54 +0300 Subject: [PATCH 276/950] fix check for nondeterministic mutations --- src/Interpreters/MutationsInterpreter.cpp | 41 ++++++++++++------- .../01162_strange_mutations.reference | 6 +-- .../0_stateless/01162_strange_mutations.sh | 20 +++++++-- 3 files changed, 44 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 540d5c76c97..b1c578854a7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -54,24 +54,33 @@ public: { ContextPtr context; std::optional nondeterministic_function_name; + bool subquery = false; }; - static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & child) + static bool needChildVisit(const ASTPtr & /*node*/, const ASTPtr & /*child*/) { - return child != nullptr; + return true; } static void visit(const ASTPtr & node, Data & data) { - if (data.nondeterministic_function_name) + if (data.nondeterministic_function_name || data.subquery) return; - if (const auto * function = typeid_cast(node.get())) + if (node->as()) + { + /// We cannot determine if subquery is deterministic or not, + /// so we do not allow to use subqueries in mutation without allow_nondeterministic_mutations=1 + data.subquery = true; + } + else if (const auto * function = typeid_cast(node.get())) { /// Property of being deterministic for lambda expression is completely determined /// by the contents of its definition, so we just proceed to it. if (function->name != "lambda") { + /// NOTE It may be an aggregate function, so get(...) may throw. + /// However, an aggregate function can be used only in subquery and we do not go into subquery. const auto func = FunctionFactory::instance().get(function->name, data.context); if (!func->isDeterministic()) data.nondeterministic_function_name = func->getName(); @@ -81,10 +90,11 @@ public: }; using FirstNonDeterministicFunctionFinder = InDepthNodeVisitor; +using FirstNonDeterministicFunctionData = FirstNonDeterministicFunctionMatcher::Data; -std::optional findFirstNonDeterministicFunctionName(const MutationCommand & command, ContextPtr context) +FirstNonDeterministicFunctionData findFirstNonDeterministicFunctionName(const MutationCommand & command, ContextPtr context) { - FirstNonDeterministicFunctionMatcher::Data finder_data{context, std::nullopt}; + FirstNonDeterministicFunctionMatcher::Data finder_data{context, std::nullopt, false}; switch (command.type) { @@ -94,7 +104,7 @@ std::optional findFirstNonDeterministicFunctionName(const MutationComman FirstNonDeterministicFunctionFinder(finder_data).visit(update_assignments_ast); if (finder_data.nondeterministic_function_name) - return finder_data.nondeterministic_function_name; + return finder_data; /// Currently UPDATE and DELETE both always have predicates so we can use fallthrough [[fallthrough]]; @@ -105,7 +115,7 @@ std::optional findFirstNonDeterministicFunctionName(const MutationComman auto predicate_ast = command.predicate->clone(); FirstNonDeterministicFunctionFinder(finder_data).visit(predicate_ast); - return finder_data.nondeterministic_function_name; + return finder_data; } default: @@ -918,12 +928,15 @@ void MutationsInterpreter::validate() { for (const auto & command : commands) { - const auto nondeterministic_func_name = findFirstNonDeterministicFunctionName(command, context); - if (nondeterministic_func_name) - throw Exception( - "ALTER UPDATE/ALTER DELETE statements must use only deterministic functions! " - "Function '" + *nondeterministic_func_name + "' is non-deterministic", - ErrorCodes::BAD_ARGUMENTS); + const auto nondeterministic_func_data = findFirstNonDeterministicFunctionName(command, context); + if (nondeterministic_func_data.subquery) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ALTER UPDATE/ALTER DELETE statement with subquery may be nondeterministic, " + "see allow_nondeterministic_mutations setting"); + + if (nondeterministic_func_data.nondeterministic_function_name) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "ALTER UPDATE/ALTER DELETE statements must use only deterministic functions. " + "Function '{}' is non-deterministic", *nondeterministic_func_data.nondeterministic_function_name); } } diff --git a/tests/queries/0_stateless/01162_strange_mutations.reference b/tests/queries/0_stateless/01162_strange_mutations.reference index 4c925f849d6..55f17cfe464 100644 --- a/tests/queries/0_stateless/01162_strange_mutations.reference +++ b/tests/queries/0_stateless/01162_strange_mutations.reference @@ -4,7 +4,6 @@ MergeTree 2 0 50 6225 0 -0 50 6225 1900 ReplicatedMergeTree 1 @@ -12,15 +11,13 @@ ReplicatedMergeTree 2 0 50 6225 0 -2 -50 6225 0 +50 6225 1900 Memory 1 2 2 0 50 6225 0 -0 50 6225 1900 Join 1 @@ -28,5 +25,4 @@ Join 2 0 50 6225 0 -0 50 6225 0 diff --git a/tests/queries/0_stateless/01162_strange_mutations.sh b/tests/queries/0_stateless/01162_strange_mutations.sh index c75bee47e4e..fecb1b8d8c0 100755 --- a/tests/queries/0_stateless/01162_strange_mutations.sh +++ b/tests/queries/0_stateless/01162_strange_mutations.sh @@ -16,17 +16,29 @@ do $CLICKHOUSE_CLIENT -q "insert into t values (1)" $CLICKHOUSE_CLIENT -q "insert into t values (2)" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select * from (select * from t where n global in (1::Int32)))" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "alter table t + delete where n global in (select * from (select * from t where n global in (1::Int32)))" $CLICKHOUSE_CLIENT -q "select * from t order by n" - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "alter table t delete where n global in (select t1.n from t as t1 full join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "alter table t + delete where n global in (select t1.n from t as t1 full join t as t2 on t1.n=t2.n where t1.n global in (select 2::Int32))" $CLICKHOUSE_CLIENT -q "select count() from t" $CLICKHOUSE_CLIENT -q "drop table t" $CLICKHOUSE_CLIENT -q "drop table if exists test" $CLICKHOUSE_CLIENT -q "CREATE TABLE test ENGINE=$engine AS SELECT number + 100 AS n, 0 AS test FROM numbers(50)" $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" - # FIXME it's not clear if the following query should fail or not - $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -c "Unknown function" + if [[ $engine == *"ReplicatedMergeTree"* ]]; then + $CLICKHOUSE_CLIENT -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "statement with subquery may be nondeterministic" + $CLICKHOUSE_CLIENT --allow_nondeterministic_mutations=1 --mutations_sync=1 -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" + elif [[ $engine == *"Join"* ]]; then + $CLICKHOUSE_CLIENT -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" 2>&1| grep -Fa "DB::Exception: " | grep -Fv "Table engine Join supports only DELETE mutations" + else + $CLICKHOUSE_CLIENT --mutations_sync=1 -q "ALTER TABLE test + UPDATE test = (SELECT groupArray(id) FROM t1 GROUP BY 1)[n - 99] WHERE 1" + fi $CLICKHOUSE_CLIENT -q "select count(), sum(n), sum(test) from test" $CLICKHOUSE_CLIENT -q "drop table test" done From 4233e2cca452a3f987e774d1066194a68769e207 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 13:23:25 +0000 Subject: [PATCH 277/950] fix --- src/Storages/FileLog/FileLogSource.cpp | 2 -- src/Storages/FileLog/StorageFileLog.cpp | 5 ++--- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 0c266178142..7beee42c99c 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -1,5 +1,3 @@ -#include -#include #include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c9b29a5a9d1..a75c9b0190e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -484,8 +483,8 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) if (file_ctx.reader) { - if ((*file_ctx.reader).is_open()) - (*file_ctx.reader).close(); + if (file_ctx.reader->is_open()) + file_ctx.reader->close(); } auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); From d24bfce93fda2d35360213adc3f90936d8cab010 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Oct 2021 17:03:54 +0300 Subject: [PATCH 278/950] Add coroutines example. --- src/Core/examples/CMakeLists.txt | 3 + src/Core/examples/coro.cpp | 202 +++++++++++++++++++++++++++++++ 2 files changed, 205 insertions(+) create mode 100644 src/Core/examples/coro.cpp diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt index 6b07dfbbfa6..c8846eb1743 100644 --- a/src/Core/examples/CMakeLists.txt +++ b/src/Core/examples/CMakeLists.txt @@ -13,3 +13,6 @@ target_link_libraries (mysql_protocol PRIVATE dbms) if(USE_SSL) target_include_directories (mysql_protocol SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) endif() + +add_executable (coro coro.cpp) +target_link_libraries (coro PRIVATE clickhouse_common_io) diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp new file mode 100644 index 00000000000..c8e2f7418e4 --- /dev/null +++ b/src/Core/examples/coro.cpp @@ -0,0 +1,202 @@ +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#if defined(__clang__) + +#include + +template +using coroutine_handle = std::experimental::coroutine_handle; + +using default_coroutine_handle = std::experimental::coroutine_handle<>; + +using suspend_never = std::experimental::suspend_never; +using suspend_always = std::experimental::suspend_always; + +#else + +#include + +template +using coroutine_handle = std::coroutine_handle; + +using default_coroutine_handle = std::coroutine_handle<>; + +using suspend_never = std::suspend_never; +using suspend_always = std::suspend_always; + +#endif + + +template +struct suspend_never_val +{ + constexpr bool await_ready() const noexcept { return true; } + constexpr void await_suspend(default_coroutine_handle) const noexcept {} + constexpr T await_resume() const noexcept + { + std::cout << " ret " << val << std::endl; + return val; + } + + T val; +}; + +template +struct resumable +{ + struct promise_type + { + using coro_handle = coroutine_handle; + auto get_return_object() { return coro_handle::from_promise(*this); } + auto initial_suspend() { return suspend_never(); } + auto final_suspend() noexcept { return suspend_never_val{*r->value}; } + //void return_void() {} + void return_value(T value_) { r->value = value_; } + void unhandled_exception() + { + DB::tryLogCurrentException("Logger"); + r->exception = std::current_exception(); + } + + explicit promise_type(std::string tag_) : tag(tag_) {} + ~promise_type() { std::cout << "~promise_type " << tag << std::endl; } + std::string tag; + coro_handle next; + resumable * r = nullptr; + }; + + using coro_handle = coroutine_handle; + + bool await_ready() const noexcept { return false; } + void await_suspend(coro_handle g) noexcept + { + std::cout << " await_suspend " << my.promise().tag << std::endl; + std::cout << " g tag " << g.promise().tag << std::endl; + g.promise().next = my; + } + T await_resume() noexcept + { + std::cout << " await_res " << my.promise().tag << std::endl; + return *value; + } + + resumable(coro_handle handle) : my(handle), tag(handle.promise().tag) + { + assert(handle); + my.promise().r = this; + std::cout << " resumable " << tag << std::endl; + } + resumable(resumable &) = delete; + resumable(resumable &&rhs) : my(rhs.my), tag(rhs.tag) + { + rhs.my = {}; + std::cout << " resumable&& " << tag << std::endl; + } + static bool resume_impl(resumable *r) + { + if (r->value) + return false; + + auto & next = r->my.promise().next; + + if (next) + { + if (resume_impl(next.promise().r)) + return true; + next = {}; + } + + if (!r->value) + { + r->my.resume(); + if (r->exception) + std::rethrow_exception(r->exception); + } + return !r->value; + } + + bool resume() + { + return resume_impl(this); + } + + T res() + { + return *value; + } + + ~resumable() + { + std::cout << " ~resumable " << tag << std::endl; + } + +private: + coro_handle my; + std::string tag; + std::optional value; + std::exception_ptr exception; +}; + +resumable boo(std::string tag) +{ + std::cout << "x" << std::endl; + co_await suspend_always(); + std::cout << StackTrace().toString(); + std::cout << "y" << std::endl; + co_return 1; +} + +resumable bar(std::string tag) +{ + std::cout << "a" << std::endl; + int res1 = co_await boo("boo1"); + std::cout << "b " << res1 << std::endl; + int res2 = co_await boo("boo2"); + if (res2 == 1) + throw DB::Exception(1, "hello"); + std::cout << "c " << res2 << std::endl; + co_return res1 + res2; // 1 + 1 = 2 +} + +resumable foo(std::string tag) { + std::cout << "Hello" << std::endl; + auto res1 = co_await bar("bar1"); + std::cout << "Coro " << res1 << std::endl; + auto res2 = co_await bar("bar2"); + std::cout << "World " << res2 << std::endl; + co_return res1 * res2; // 2 * 2 = 4 +} + +int main() +{ + Poco::AutoPtr app_channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(app_channel); + Poco::Logger::root().setLevel("trace"); + + LOG_INFO(&Poco::Logger::get(""), "Starting"); + + try + { + auto t = foo("foo"); + std::cout << ".. started" << std::endl; + while (t.resume()) + std::cout << ".. yielded" << std::endl; + std::cout << ".. done: " << t.res() << std::endl; + } + catch (DB::Exception & e) + { + std::cout << "Got exception " << e.what() << std::endl; + std::cout << e.getStackTraceString() << std::endl; + } +} From 3a3ea9f19cf1fa3fab45b01a2630f42f0bef145b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 7 Oct 2021 18:42:25 +0300 Subject: [PATCH 279/950] Update 00652_replicated_mutations_default_database_zookeeper.sh --- .../00652_replicated_mutations_default_database_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh index 3f5b8d570a6..0ac5a2f748a 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_default_database_zookeeper.sh @@ -9,7 +9,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --multiquery << EOF +${CLICKHOUSE_CLIENT} --allow_nondeterministic_mutations=1 --multiquery << EOF DROP TABLE IF EXISTS mutations_r1; DROP TABLE IF EXISTS for_subquery; From 00fbf48a683ec009b26f0c47f931c7441013dbbe Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 7 Oct 2021 19:09:40 +0300 Subject: [PATCH 280/950] Minor fixes. --- docs/en/sql-reference/functions/geo/h3.md | 14 +++++++------- docs/ru/sql-reference/functions/geo/h3.md | 23 ++++++++++++----------- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 410cb9d3cc2..048834806d1 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -40,7 +40,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT h3IsValid(630814730351855103) as h3IsValid; +SELECT h3IsValid(630814730351855103) AS h3IsValid; ``` Result: @@ -77,7 +77,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT h3GetResolution(639821929606596015) as resolution; +SELECT h3GetResolution(639821929606596015) AS resolution; ``` Result: @@ -111,7 +111,7 @@ h3EdgeAngle(resolution) Query: ``` sql -SELECT h3EdgeAngle(10) as edgeAngle; +SELECT h3EdgeAngle(10) AS edgeAngle; ``` Result: @@ -145,7 +145,7 @@ h3EdgeLengthM(resolution) Query: ``` sql -SELECT h3EdgeLengthM(15) as edgeLengthM; +SELECT h3EdgeLengthM(15) AS edgeLengthM; ``` Result: @@ -184,7 +184,7 @@ Type: [UInt64](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index; +SELECT geoToH3(37.79506683, 55.71290588, 15) AS h3Index; ``` Result: @@ -333,7 +333,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT h3GetBaseCell(612916788725809151) as basecell; +SELECT h3GetBaseCell(612916788725809151) AS basecell; ``` Result: @@ -369,7 +369,7 @@ Type: [Float64](../../../sql-reference/data-types/float.md). Query: ``` sql -SELECT h3HexAreaM2(13) as area; +SELECT h3HexAreaM2(13) AS area; ``` Result: diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md index 6bc6943ec93..e85236848f6 100644 --- a/docs/ru/sql-reference/functions/geo/h3.md +++ b/docs/ru/sql-reference/functions/geo/h3.md @@ -38,7 +38,7 @@ h3IsValid(h3index) Запрос: ``` sql -SELECT h3IsValid(630814730351855103) as h3IsValid; +SELECT h3IsValid(630814730351855103) AS h3IsValid; ``` Результат: @@ -75,7 +75,7 @@ h3GetResolution(h3index) Запрос: ``` sql -SELECT h3GetResolution(639821929606596015) as resolution; +SELECT h3GetResolution(639821929606596015) AS resolution; ``` Результат: @@ -109,7 +109,7 @@ h3EdgeAngle(resolution) Запрос: ``` sql -SELECT h3EdgeAngle(10) as edgeAngle; +SELECT h3EdgeAngle(10) AS edgeAngle; ``` Результат: @@ -143,7 +143,7 @@ h3EdgeLengthM(resolution) Запрос: ``` sql -SELECT h3EdgeLengthM(15) as edgeLengthM; +SELECT h3EdgeLengthM(15) AS edgeLengthM; ``` Результат: @@ -182,7 +182,7 @@ geoToH3(lon, lat, resolution) Запрос: ``` sql -SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index; +SELECT geoToH3(37.79506683, 55.71290588, 15) AS h3Index; ``` Результат: @@ -295,7 +295,7 @@ h3GetBaseCell(index) Запрос: ``` sql -SELECT h3GetBaseCell(612916788725809151) as basecell; +SELECT h3GetBaseCell(612916788725809151) AS basecell; ``` Результат: @@ -329,7 +329,7 @@ h3HexAreaM2(resolution) Запрос: ``` sql -SELECT h3HexAreaM2(13) as area; +SELECT h3HexAreaM2(13) AS area; ``` Результат: @@ -441,7 +441,7 @@ h3ToParent(index, resolution) Запрос: ``` sql -SELECT h3ToParent(599405990164561919, 3) as parent; +SELECT h3ToParent(599405990164561919, 3) AS parent; ``` Результат: @@ -475,7 +475,7 @@ h3ToString(index) Запрос: ``` sql -SELECT h3ToString(617420388352917503) as h3_string; +SELECT h3ToString(617420388352917503) AS h3_string; ``` Результат: @@ -512,7 +512,7 @@ stringToH3(index_str) Запрос: ``` sql -SELECT stringToH3('89184926cc3ffff') as index; +SELECT stringToH3('89184926cc3ffff') AS index; ``` Результат: @@ -667,5 +667,6 @@ SELECT h3GetFaces(599686042433355775) AS faces; ┌─faces─┐ │ [7] │ └───────┘ +``` -[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3/) +[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3) From 1df9afb47cf5204be24edbe0e8c8c631ea1e759f Mon Sep 17 00:00:00 2001 From: michon470 <71978106+michon470@users.noreply.github.com> Date: Thu, 7 Oct 2021 22:27:22 +0300 Subject: [PATCH 281/950] Update docs/en/sql-reference/statements/alter/column.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 31874ef208d..ef4b88af6ba 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -204,7 +204,7 @@ Syntax: ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; ``` -**Example with the creation of new column:** +**Example** ```sql DROP TABLE IF EXISTS tmp; From fc0bccb0c6f6fec55800235fde76ef6669c5b5f9 Mon Sep 17 00:00:00 2001 From: michon470 <71978106+michon470@users.noreply.github.com> Date: Thu, 7 Oct 2021 22:27:39 +0300 Subject: [PATCH 282/950] Update docs/ru/sql-reference/statements/alter/column.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 366caf6a2a0..5ab7207c580 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -196,7 +196,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ## MATERIALIZE COLUMN {#materialize-column} -С помощью этого запроса можно сделать столбец таблицы материализованным (`MATERIALIZED`) в его частях, у которых отсутствуют значения. Это полезно, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`). Если вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо большим, для него может оказаться целесообразным использовать `MATERIALIZE COLUMN`. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`, как показано ниже. +Материализует столбец таблицы в кусках, в которых отсутствуют значения. Используется, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`), потому как вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо затратным. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`. Синтаксис: From 0eaf2f12a31e1ed0f9dff5bfcd2059123541603f Mon Sep 17 00:00:00 2001 From: michon470 <71978106+michon470@users.noreply.github.com> Date: Thu, 7 Oct 2021 22:27:46 +0300 Subject: [PATCH 283/950] Update docs/ru/sql-reference/statements/alter/column.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 5ab7207c580..c6269f0eb62 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -204,7 +204,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; ``` -**Пример использования при создании нового столбца:** +**Пример** ```sql DROP TABLE IF EXISTS tmp; From c030756c38f75deaf0c3fd9e00c762e376d515c3 Mon Sep 17 00:00:00 2001 From: michon470 <71978106+michon470@users.noreply.github.com> Date: Thu, 7 Oct 2021 22:28:02 +0300 Subject: [PATCH 284/950] Update docs/en/sql-reference/statements/alter/column.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index ef4b88af6ba..aee3823bc05 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -196,7 +196,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL; ## MATERIALIZE COLUMN {#materialize-column} -Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier as shown below. +Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier. Syntax: From 76e3ef686a244d13cbc37249ae260873ed36fae2 Mon Sep 17 00:00:00 2001 From: michon470 <71978106+michon470@users.noreply.github.com> Date: Thu, 7 Oct 2021 22:28:12 +0300 Subject: [PATCH 285/950] Update docs/ru/sql-reference/statements/alter/column.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index c6269f0eb62..ef3d98fc10e 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -25,7 +25,7 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|MODIFY| - [COMMENT COLUMN](#alter_comment-column) — добавляет комментарий к столбцу; - [MODIFY COLUMN](#alter_modify-column) — изменяет тип столбца, выражение для значения по умолчанию и TTL; - [MODIFY COLUMN REMOVE](#modify-remove) — удаляет какое-либо из свойств столбца; -- [MATERIALIZE COLUMN](#materialize-column) — делает столбец материализованным (MATERIALIZED) в его частях, у которых отсутствуют значения. +- [MATERIALIZE COLUMN](#materialize-column) — делает столбец материализованным (`MATERIALIZED`) в кусках, в которых отсутствуют значения. Подробное описание для каждого действия приведено ниже. From 9389cb7c7702574dcf6224ef0e7c4d83e7a30896 Mon Sep 17 00:00:00 2001 From: Mikhail <71978106+michon470@users.noreply.github.com> Date: Thu, 7 Oct 2021 22:31:44 +0300 Subject: [PATCH 286/950] Example corrected --- docs/en/sql-reference/statements/alter/column.md | 8 ++++---- docs/ru/sql-reference/statements/alter/column.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index aee3823bc05..4eb251b88cd 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -210,7 +210,7 @@ ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; DROP TABLE IF EXISTS tmp; SET mutations_sync = 2; CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple(); -INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20; +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 10; ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); SELECT groupArray(x), groupArray(s) FROM tmp; ``` @@ -218,9 +218,9 @@ SELECT groupArray(x), groupArray(s) FROM tmp; **Result:** ```sql -┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │ -└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘ +┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','5','6','7','8','9'] │ +└───────────────────────┴───────────────────────────────────────────┘ ``` **See Also** diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index ef3d98fc10e..bfd52801210 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -210,7 +210,7 @@ ALTER TABLE table MATERIALIZE COLUMN col [FINAL]; DROP TABLE IF EXISTS tmp; SET mutations_sync = 2; CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple(); -INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20; +INSERT INTO tmp SELECT * FROM system.numbers LIMIT 10; ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x); SELECT groupArray(x), groupArray(s) FROM tmp; ``` @@ -218,9 +218,9 @@ SELECT groupArray(x), groupArray(s) FROM tmp; **Результат:** ```sql -┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐ -│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │ -└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘ +┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────┐ +│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','5','6','7','8','9'] │ +└───────────────────────┴───────────────────────────────────────────┘ ``` ## Ограничения запроса ALTER {#ogranicheniia-zaprosa-alter} From e07a6f3fc0ea0b496483287d85b50d29f5a8c330 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Oct 2021 21:09:35 +0300 Subject: [PATCH 287/950] docker: add pandas/clickhouse_driver into test images --- docker/test/fasttest/Dockerfile | 2 +- docker/test/fuzzer/Dockerfile | 2 +- docker/test/stateless/Dockerfile | 2 +- docker/test/style/Dockerfile | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 798910fb952..f50c65bb9f2 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -67,7 +67,7 @@ RUN apt-get update \ unixodbc \ --yes --no-install-recommends -RUN pip3 install numpy scipy pandas Jinja2 +RUN pip3 install numpy scipy pandas Jinja2 pandas clickhouse_driver # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index 6444e745c47..13353bc2960 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -27,7 +27,7 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -RUN pip3 install Jinja2 +RUN pip3 install Jinja2 pandas clickhouse_driver COPY * / diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 7de8c061673..a5733d11dd2 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -34,7 +34,7 @@ RUN apt-get update -y \ postgresql-client \ sqlite3 -RUN pip3 install numpy scipy pandas Jinja2 +RUN pip3 install numpy scipy pandas Jinja2 clickhouse_driver RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 33cdb9db57a..64cc0c9c7b7 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -10,7 +10,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ pylint \ yamllint \ - && pip3 install codespell + && pip3 install codespell pandas clickhouse_driver COPY run.sh / COPY process_style_check_result.py / From 9dd0fca1edd383c00667ce4c1a953e4f6d2bca1a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Oct 2021 23:45:18 +0300 Subject: [PATCH 288/950] Suppress some existed warnings in clickhouse-test (will be fixed separately) --- tests/clickhouse-test | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 19080f3934f..2c8093190ea 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1,6 +1,12 @@ #!/usr/bin/env python3 # pylint: disable=too-many-return-statements +# pylint: disable=consider-using-f-string +# pylint: disable=global-variable-not-assigned +# pylint: disable=consider-using-with +# pylint: disable=unspecified-encoding +# pylint: disable=consider-using-min-builtin + import enum import shutil import sys From df129d7efc70eb2abc394b72a0dd64c421de8549 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 7 Oct 2021 21:05:42 +0300 Subject: [PATCH 289/950] Rewrite clickhouse-test to use python clickhouse_driver Pros: - Using native protocol over executing binaries is always better - `clickhouse-client` in debug build takes almost a second to execute simple `SELECT 1` and `clickhouse-test` requires ~5 queries at start (determine some flags, zk, alive, create database) Notes: - `FORMAT Vertical` had been replaced with printing of `pandas.DataFrame` And after this patch tiny tests work with the speed of the test, and does not requires +-5 seconds of bootstrapping. --- tests/clickhouse-test | 424 +++++++++++++++++++----------------------- 1 file changed, 193 insertions(+), 231 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2c8093190ea..e8c85a6ae79 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -19,13 +19,10 @@ import traceback import math from argparse import ArgumentParser -from typing import Tuple, Union, Optional, TextIO, Dict, Set, List -import shlex +from typing import Tuple, Union, Optional, Dict, Set, List import subprocess from subprocess import Popen from subprocess import PIPE -from subprocess import CalledProcessError -from subprocess import TimeoutExpired from datetime import datetime from time import time, sleep from errno import ESRCH @@ -41,6 +38,9 @@ import multiprocessing import socket from contextlib import closing +import clickhouse_driver +import pandas + USE_JINJA = True try: import jinja2 @@ -48,20 +48,45 @@ except ImportError: USE_JINJA = False print('WARNING: jinja2 not installed! Template tests will be skipped.') -DISTRIBUTED_DDL_TIMEOUT_MSG = "is executing longer than distributed_ddl_task_timeout" - MESSAGES_TO_RETRY = [ "ConnectionPoolWithFailover: Connection failed at try", "DB::Exception: New table appeared in database being dropped or detached. Try again", "is already started to be removing by another replica right now", "DB::Exception: Cannot enqueue query", - DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME + "is executing longer than distributed_ddl_task_timeout" # FIXME +] +error_codes = clickhouse_driver.errors.ErrorCodes +error_codes.NOT_A_LEADER = 529 +ERROR_CODES_TO_RETRY = [ + error_codes.ALL_CONNECTION_TRIES_FAILED, + error_codes.DATABASE_NOT_EMPTY, + error_codes.NOT_A_LEADER, + error_codes.UNFINISHED, ] MAX_RETRIES = 3 TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect'] +class Client(clickhouse_driver.Client): + # return first column of the first row + def execute_one(self, *args, **kwargs): + return super().execute(*args, **kwargs)[0][0] + + # return pandas.DataFrame + def execute_pandas(self, *args, **kwargs): + data = super().execute(*args, **kwargs, with_column_types=True) + return Client.__combine(data) + + @staticmethod + def __combine(data): + cols = data[1] + rows = data[0] + header = [ i[0] for i in cols ] + data = pandas.DataFrame(data=rows, columns=header) + return data + + class Terminated(KeyboardInterrupt): pass @@ -103,18 +128,16 @@ def get_db_engine(args, database_name): def get_zookeeper_session_uptime(args): + global clickhouse_client + try: - query = b"SELECT zookeeperSessionUptime()" - if args.replicated_database: - query = b"SELECT min(materialize(zookeeperSessionUptime())) " \ - b"FROM clusterAllReplicas('test_cluster_database_replicated', system.one) " - - clickhouse_proc = open_client_process(args.client) - - (stdout, _) = clickhouse_proc.communicate((query), timeout=20) - - return int(stdout.decode('utf-8').strip()) + return int(clickhouse_client.execute_one(""" + SELECT min(materialize(zookeeperSessionUptime())) + FROM clusterAllReplicas('test_cluster_database_replicated', system.one) + """)) + else: + return int(clickhouse_client.execute_one('SELECT zookeeperSessionUptime()')) except: return None @@ -128,24 +151,31 @@ def need_retry(args, stdout, stderr, total_time): return True return any(msg in stdout for msg in MESSAGES_TO_RETRY) or any(msg in stderr for msg in MESSAGES_TO_RETRY) +def need_retry_error(args, error, total_time): + # Sometimes we may get unexpected exception like "Replica is readonly" or "Shutdown is called for table" + # instead of "Session expired" or "Connection loss" + # Retry if session was expired during test execution + session_uptime = get_zookeeper_session_uptime(args) + if session_uptime is not None and session_uptime < math.ceil(total_time): + return True + if isinstance(error, clickhouse_driver.errors.Error): + if error.code in ERROR_CODES_TO_RETRY: + return True + if any(msg in error.message for msg in MESSAGES_TO_RETRY): + return True + return False + def get_processlist(args): - try: - query = b"SHOW PROCESSLIST FORMAT Vertical" - - if args.replicated_database: - query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ - b"FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) " \ - b"WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" - - clickhouse_proc = open_client_process(args.client) - - (stdout, _) = clickhouse_proc.communicate((query), timeout=20) - - return False, stdout.decode('utf-8') - except Exception as ex: - print("Exception", ex) - return True, "" + global clickhouse_client + if args.replicated_database: + return clickhouse_client.execute_pandas(""" + SELECT materialize((hostName(), tcpPort())) as host, * + FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) + WHERE query NOT LIKE '%system.processes%' + """) + else: + return clickhouse_client.execute_pandas('SHOW PROCESSLIST') # collect server stacktraces using gdb @@ -311,7 +341,8 @@ class TestCase: return None @staticmethod - def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file): + def configure_testcase_args(args, case_file, suite_tmp_dir): + global clickhouse_client testcase_args = copy.deepcopy(args) testcase_args.testcase_start_time = datetime.now() @@ -331,23 +362,11 @@ class TestCase: database = 'test_{suffix}'.format(suffix=random_str()) - with open(stderr_file, 'w') as stderr: - client_cmd = testcase_args.testcase_client + " " \ - + get_additional_client_options(args) - - clickhouse_proc_create = open_client_process( - universal_newlines=True, - client_args=client_cmd, - stderr_file=stderr) - - try: - clickhouse_proc_create.communicate( - ("CREATE DATABASE " + database + get_db_engine(testcase_args, database)), - timeout=testcase_args.timeout) - except TimeoutExpired: - total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() - return clickhouse_proc_create, "", "Timeout creating database {} before test".format( - database), total_time + try: + clickhouse_client.execute("CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename}) + except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError): + total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() + return None, "", f"Timeout creating database {database} before test", total_time os.environ["CLICKHOUSE_DATABASE"] = database # Set temporary directory to match the randomly generated database, @@ -418,41 +437,42 @@ class TestCase: def process_result_impl(self, proc, stdout: str, stderr: str, total_time: float): description = "" - if proc.returncode is None: - try: - proc.kill() - except OSError as e: - if e.errno != ESRCH: - raise + if proc: + if proc.returncode is None: + try: + proc.kill() + except OSError as e: + if e.errno != ESRCH: + raise - if stderr: - description += stderr - return TestResult(self.name, TestStatus.FAIL, FailureReason.TIMEOUT, total_time, description) + if stderr: + description += stderr + return TestResult(self.name, TestStatus.FAIL, FailureReason.TIMEOUT, total_time, description) - if proc.returncode != 0: - reason = FailureReason.EXIT_CODE - description += str(proc.returncode) + if proc.returncode != 0: + reason = FailureReason.EXIT_CODE + description += str(proc.returncode) - if stderr: - description += "\n" - description += stderr + if stderr: + description += "\n" + description += stderr - # Stop on fatal errors like segmentation fault. They are sent to client via logs. - if ' ' in stderr: - reason = FailureReason.SERVER_DIED + # Stop on fatal errors like segmentation fault. They are sent to client via logs. + if ' ' in stderr: + reason = FailureReason.SERVER_DIED - if self.testcase_args.stop \ - and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) \ - and 'Received exception from server' not in stderr: - reason = FailureReason.SERVER_DIED + if self.testcase_args.stop \ + and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) \ + and 'Received exception from server' not in stderr: + reason = FailureReason.SERVER_DIED - if os.path.isfile(self.stdout_file): - description += ", result:\n\n" - description += '\n'.join(open(self.stdout_file).read().split('\n')[:100]) - description += '\n' + if os.path.isfile(self.stdout_file): + description += ", result:\n\n" + description += '\n'.join(open(self.stdout_file).read().split('\n')[:100]) + description += '\n' - description += "\nstdout:\n{}\n".format(stdout) - return TestResult(self.name, TestStatus.FAIL, reason, total_time, description) + description += "\nstdout:\n{}\n".format(stdout) + return TestResult(self.name, TestStatus.FAIL, reason, total_time, description) if stderr: description += "\n{}\n".format('\n'.join(stderr.split('\n')[:100])) @@ -516,21 +536,12 @@ class TestCase: @staticmethod def send_test_name_failed(suite: str, case: str) -> bool: - clickhouse_proc = open_client_process(args.client, universal_newlines=True) - - failed_to_check = False - + global clickhouse_client pid = os.getpid() - query = f"SELECT 'Running test {suite}/{case} from pid={pid}';" - - try: - clickhouse_proc.communicate((query), timeout=20) - except: - failed_to_check = True - - return failed_to_check or clickhouse_proc.returncode != 0 + clickhouse_client.execute(f"SELECT 'Running test {suite}/{case} from pid={pid}'") def run_single_test(self, server_logs_level, client_options): + global clickhouse_client args = self.testcase_args client = args.testcase_client start_time = args.testcase_start_time @@ -572,28 +583,13 @@ class TestCase: need_drop_database = not maybe_passed if need_drop_database: - with open(self.stderr_file, 'a') as stderr: - clickhouse_proc_create = open_client_process(client, universal_newlines=True, stderr_file=stderr) - seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) - try: - drop_database_query = "DROP DATABASE " + database - if args.replicated_database: - drop_database_query += " ON CLUSTER test_cluster_database_replicated" - clickhouse_proc_create.communicate((drop_database_query), timeout=seconds_left) - except TimeoutExpired: - # kill test process because it can also hung - if proc.returncode is None: - try: - proc.kill() - except OSError as e: - if e.errno != ESRCH: - raise - + with clickhouse_client.connection.timeout_setter(seconds_left): + clickhouse_client.execute("DROP DATABASE " + database) + except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError): total_time = (datetime.now() - start_time).total_seconds() - return clickhouse_proc_create, "", f"Timeout dropping database {database} after test", total_time - + return None, "", f"Timeout dropping database {database} after test", total_time shutil.rmtree(args.test_tmp_dir) total_time = (datetime.now() - start_time).total_seconds() @@ -624,12 +620,15 @@ class TestCase: if skip_reason is not None: return TestResult(self.name, TestStatus.SKIPPED, skip_reason, 0., "") - if args.testname and self.send_test_name_failed(suite.suite, self.case): - description = "\nServer does not respond to health check\n" - return TestResult(self.name, TestStatus.FAIL, FailureReason.SERVER_DIED, 0., description) + if args.testname: + try: + self.send_test_name_failed(suite.suite, self.case) + except: + return TestResult(self.name, TestStatus.FAIL, FailureReason.SERVER_DIED, 0., + "\nServer does not respond to health check\n") self.runs_count += 1 - self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path, self.stderr_file) + self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path) proc, stdout, stderr, total_time = self.run_single_test(server_logs_level, client_options) result = self.process_result_impl(proc, stdout, stderr, total_time) @@ -794,12 +793,8 @@ class TestSuite: @staticmethod def readTestSuite(args, suite_dir_name: str): def is_data_present(): - clickhouse_proc = open_client_process(args.client) - (stdout, stderr) = clickhouse_proc.communicate(b"EXISTS TABLE test.hits") - if clickhouse_proc.returncode != 0: - raise CalledProcessError(clickhouse_proc.returncode, args.client, stderr) - - return stdout.startswith(b'1') + global clickhouse_client + return int(clickhouse_client.execute_one('EXISTS TABLE test.hits')) base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) @@ -832,6 +827,7 @@ class TestSuite: stop_time = None +clickhouse_client = None exit_code = None server_died = None stop_tests_triggered_lock = None @@ -961,42 +957,26 @@ def run_tests_array(all_tests_with_params): server_logs_level = "warning" -def check_server_started(client, retry_count): +def check_server_started(retry_count): + global clickhouse_client print("Connecting to ClickHouse server...", end='') sys.stdout.flush() - while retry_count > 0: - clickhouse_proc = open_client_process(client) - (stdout, stderr) = clickhouse_proc.communicate(b"SELECT 1") - - if clickhouse_proc.returncode == 0 and stdout.startswith(b"1"): + try: + clickhouse_client.execute('SELECT 1') print(" OK") sys.stdout.flush() return True - - if clickhouse_proc.returncode == 210: - # Connection refused, retry + except (ConnectionRefusedError, ConnectionResetError, clickhouse_driver.errors.NetworkError): print('.', end='') sys.stdout.flush() retry_count -= 1 sleep(0.5) continue - code: int = clickhouse_proc.returncode - - print(f"\nClient invocation failed with code {code}:\n\ - stdout: {stdout}\n\ - stderr: {stderr}\n\ - args: {''.join(clickhouse_proc.args)}\n") - - sys.stdout.flush() - - return False - print('\nAll connection tries failed') sys.stdout.flush() - return False @@ -1012,60 +992,38 @@ class BuildFlags(): POLYMORPHIC_PARTS = 'polymorphic-parts' -def collect_build_flags(client): - clickhouse_proc = open_client_process(client) - (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") +def collect_build_flags(): + global clickhouse_client + result = [] - if clickhouse_proc.returncode == 0: - if b'-fsanitize=thread' in stdout: - result.append(BuildFlags.THREAD) - elif b'-fsanitize=address' in stdout: - result.append(BuildFlags.ADDRESS) - elif b'-fsanitize=undefined' in stdout: - result.append(BuildFlags.UNDEFINED) - elif b'-fsanitize=memory' in stdout: - result.append(BuildFlags.MEMORY) - else: - raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") + if '-fsanitize=thread' in value: + result.append(BuildFlags.THREAD) + elif '-fsanitize=address' in value: + result.append(BuildFlags.ADDRESS) + elif '-fsanitize=undefined' in value: + result.append(BuildFlags.UNDEFINED) + elif '-fsanitize=memory' in value: + result.append(BuildFlags.MEMORY) - clickhouse_proc = open_client_process(client) - (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") + value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") + if 'Debug' in value: + result.append(BuildFlags.DEBUG) + elif 'RelWithDebInfo' in value or 'Release' in value: + result.append(BuildFlags.RELEASE) - if clickhouse_proc.returncode == 0: - if b'Debug' in stdout: - result.append(BuildFlags.DEBUG) - elif b'RelWithDebInfo' in stdout or b'Release' in stdout: - result.append(BuildFlags.RELEASE) - else: - raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") + if value in ('ON', '1'): + result.append(BuildFlags.UNBUNDLED) - clickhouse_proc = open_client_process(client) - (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") + value = clickhouse_client.execute_one("SELECT value FROM system.settings WHERE name = 'default_database_engine'") + if value == 'Ordinary': + result.append(BuildFlags.ORDINARY_DATABASE) - if clickhouse_proc.returncode == 0: - if b'ON' in stdout or b'1' in stdout: - result.append(BuildFlags.UNBUNDLED) - else: - raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - - clickhouse_proc = open_client_process(client) - (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.settings WHERE name = 'default_database_engine'") - - if clickhouse_proc.returncode == 0: - if b'Ordinary' in stdout: - result.append(BuildFlags.ORDINARY_DATABASE) - else: - raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) - - clickhouse_proc = open_client_process(client) - (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'") - - if clickhouse_proc.returncode == 0: - if stdout == b'0\n': - result.append(BuildFlags.POLYMORPHIC_PARTS) - else: - raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) + value = int(clickhouse_client.execute_one("SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")) + if value == 0: + result.append(BuildFlags.POLYMORPHIC_PARTS) return result @@ -1092,16 +1050,6 @@ def extract_key(key: str) -> str: args.configserver + key)[1] -def open_client_process( - client_args: str, - universal_newlines: bool = False, - stderr_file: Optional[TextIO] = None): - return Popen( - shlex.split(client_args), stdin=PIPE, stdout=PIPE, - stderr=stderr_file if stderr_file is not None else PIPE, - universal_newlines=True if universal_newlines else None) - - def do_run_tests(jobs, test_suite: TestSuite, parallel): if jobs > 1 and len(test_suite.parallel_tests) > 0: print("Found", len(test_suite.parallel_tests), "parallel tests and", len(test_suite.sequential_tests), "sequential tests") @@ -1170,8 +1118,9 @@ def main(args): global exit_code global server_logs_level global restarted_tests + global clickhouse_client - if not check_server_started(args.client, args.server_check_retries): + if not check_server_started(args.server_check_retries): msg = "Server is not responding. Cannot execute 'SELECT 1' query. \ If you are using split build, you have to specify -c option." if args.hung_check: @@ -1181,13 +1130,12 @@ def main(args): print_stacktraces() raise Exception(msg) - args.build_flags = collect_build_flags(args.client) + args.build_flags = collect_build_flags() if args.skip: args.skip = set(args.skip) base_dir = os.path.abspath(args.queries) - tmp_dir = os.path.abspath(args.tmp) # Keep same default values as in queries/shell_config.sh os.environ.setdefault("CLICKHOUSE_BINARY", args.binary) @@ -1218,17 +1166,12 @@ def main(args): create_database_retries = 0 while create_database_retries < MAX_RETRIES: start_time = datetime.now() - - client_cmd = args.client + " " + get_additional_client_options(args) - - clickhouse_proc_create = open_client_process(client_cmd, universal_newlines=True) - - (stdout, stderr) = clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))) - - total_time = (datetime.now() - start_time).total_seconds() - - if not need_retry(args, stdout, stderr, total_time): - break + try: + clickhouse_client.execute("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name)) + except Exception as e: + total_time = (datetime.now() - start_time).total_seconds() + if not need_retry_error(args, e, total_time): + break create_database_retries += 1 if args.database and args.database != "test": @@ -1255,18 +1198,14 @@ def main(args): # Some queries may execute in background for some time after test was finished. This is normal. for _ in range(1, 60): - timeout, processlist = get_processlist(args) - if timeout or not processlist: + processlist = get_processlist(args) + if processlist.empty: break sleep(1) - if timeout or processlist: - if processlist: - print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) - print(processlist) - else: - print(colored("Seems like server hung and cannot respond to queries", args, "red", attrs=["bold"])) - + if not processlist.empty: + print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) + print(processlist) print_stacktraces() exit_code.value = 1 @@ -1311,16 +1250,20 @@ def find_binary(name): def get_additional_client_options(args): if args.client_option: return ' '.join('--' + option for option in args.client_option) - return '' - def get_additional_client_options_url(args): if args.client_option: return '&'.join(args.client_option) - return '' +def get_additional_client_options_dict(args): + settings = {} + if args.client_option: + for key, value in map(lambda x: x.split('='), args.client_option): + settings[key] = value + return settings + if __name__ == '__main__': stop_time = None @@ -1439,14 +1382,24 @@ if __name__ == '__main__': if args.configclient: args.client += ' --config-file=' + args.configclient - if os.getenv("CLICKHOUSE_HOST"): - args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST") + tcp_host = os.getenv("CLICKHOUSE_HOST") + if tcp_host is not None: + args.client += f' --host={tcp_host}' + else: + tcp_host = 'localhost' - args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", "9000")) - args.client += f" --port={args.tcp_port}" + tcp_port = os.getenv("CLICKHOUSE_PORT_TCP") + if tcp_port is not None: + args.tcp_port = int(tcp_port) + args.client += f" --port={tcp_port}" + else: + args.tcp_port = 9000 - if os.getenv("CLICKHOUSE_DATABASE"): - args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE") + client_database = os.getenv("CLICKHOUSE_DATABASE") + if client_database is not None: + args.client += f' --database={client_database}' + else: + client_database = 'default' if args.client_option: # Set options for client @@ -1474,4 +1427,13 @@ if __name__ == '__main__': if args.jobs is None: args.jobs = multiprocessing.cpu_count() + # configure pandas to make it more like Vertical format + pandas.options.display.max_columns = None + pandas.options.display.width = None + + clickhouse_client = Client(host=tcp_host, + port=args.tcp_port, + database=client_database, + settings=get_additional_client_options_dict(args)) + main(args) From e2d6698244d43979b3fe2478dfdcd8dc3a91a0fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Oct 2021 00:07:05 +0300 Subject: [PATCH 290/950] clickhouse-test: do not use persistent connection for simplicity (due to threads) --- tests/clickhouse-test | 77 +++++++++++++++++++++---------------------- 1 file changed, 38 insertions(+), 39 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index e8c85a6ae79..6bbfa97ab66 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -86,6 +86,17 @@ class Client(clickhouse_driver.Client): data = pandas.DataFrame(data=rows, columns=header) return data +# Helpers +def make_clickhouse_client(base_args, *args, **kwargs): + return Client(host=base_args.tcp_host, port=base_args.tcp_port, + settings=get_additional_client_options_dict(base_args)) +def clickhouse_execute_one(base_args, *args, **kwargs): + return make_clickhouse_client(base_args).execute_one(*args, **kwargs) +def clickhouse_execute(base_args, *args, **kwargs): + return make_clickhouse_client(base_args).execute(*args, **kwargs) +def clickhouse_execute_pandas(base_args, *args, **kwargs): + return make_clickhouse_client(base_args).execute_pandas(*args, **kwargs) + class Terminated(KeyboardInterrupt): pass @@ -128,16 +139,14 @@ def get_db_engine(args, database_name): def get_zookeeper_session_uptime(args): - global clickhouse_client - try: if args.replicated_database: - return int(clickhouse_client.execute_one(""" + return int(clickhouse_execute_one(args, """ SELECT min(materialize(zookeeperSessionUptime())) FROM clusterAllReplicas('test_cluster_database_replicated', system.one) """)) else: - return int(clickhouse_client.execute_one('SELECT zookeeperSessionUptime()')) + return int(clickhouse_execute_one(args, 'SELECT zookeeperSessionUptime()')) except: return None @@ -167,15 +176,14 @@ def need_retry_error(args, error, total_time): def get_processlist(args): - global clickhouse_client if args.replicated_database: - return clickhouse_client.execute_pandas(""" + return clickhouse_execute_pandas(args, """ SELECT materialize((hostName(), tcpPort())) as host, * FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) WHERE query NOT LIKE '%system.processes%' """) else: - return clickhouse_client.execute_pandas('SHOW PROCESSLIST') + return clickhouse_execute_pandas(args, 'SHOW PROCESSLIST') # collect server stacktraces using gdb @@ -342,7 +350,6 @@ class TestCase: @staticmethod def configure_testcase_args(args, case_file, suite_tmp_dir): - global clickhouse_client testcase_args = copy.deepcopy(args) testcase_args.testcase_start_time = datetime.now() @@ -363,7 +370,7 @@ class TestCase: database = 'test_{suffix}'.format(suffix=random_str()) try: - clickhouse_client.execute("CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename}) + clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename}) except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError): total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() return None, "", f"Timeout creating database {database} before test", total_time @@ -536,12 +543,10 @@ class TestCase: @staticmethod def send_test_name_failed(suite: str, case: str) -> bool: - global clickhouse_client pid = os.getpid() - clickhouse_client.execute(f"SELECT 'Running test {suite}/{case} from pid={pid}'") + clickhouse_execute(args, f"SELECT 'Running test {suite}/{case} from pid={pid}'") def run_single_test(self, server_logs_level, client_options): - global clickhouse_client args = self.testcase_args client = args.testcase_client start_time = args.testcase_start_time @@ -585,8 +590,10 @@ class TestCase: if need_drop_database: seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) try: - with clickhouse_client.connection.timeout_setter(seconds_left): - clickhouse_client.execute("DROP DATABASE " + database) + client = make_clickhouse_client(args) + client.connection.force_connect() + with client.connection.timeout_setter(seconds_left): + client.execute("DROP DATABASE " + database) except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError): total_time = (datetime.now() - start_time).total_seconds() return None, "", f"Timeout dropping database {database} after test", total_time @@ -793,8 +800,7 @@ class TestSuite: @staticmethod def readTestSuite(args, suite_dir_name: str): def is_data_present(): - global clickhouse_client - return int(clickhouse_client.execute_one('EXISTS TABLE test.hits')) + return int(clickhouse_execute_one(args, 'EXISTS TABLE test.hits')) base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) @@ -827,7 +833,6 @@ class TestSuite: stop_time = None -clickhouse_client = None exit_code = None server_died = None stop_tests_triggered_lock = None @@ -957,14 +962,14 @@ def run_tests_array(all_tests_with_params): server_logs_level = "warning" -def check_server_started(retry_count): - global clickhouse_client +def check_server_started(args): print("Connecting to ClickHouse server...", end='') sys.stdout.flush() + retry_count = args.server_check_retries while retry_count > 0: try: - clickhouse_client.execute('SELECT 1') + clickhouse_execute(args, 'SELECT 1') print(" OK") sys.stdout.flush() return True @@ -992,12 +997,10 @@ class BuildFlags(): POLYMORPHIC_PARTS = 'polymorphic-parts' -def collect_build_flags(): - global clickhouse_client - +def collect_build_flags(args): result = [] - value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") + value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") if '-fsanitize=thread' in value: result.append(BuildFlags.THREAD) elif '-fsanitize=address' in value: @@ -1007,21 +1010,21 @@ def collect_build_flags(): elif '-fsanitize=memory' in value: result.append(BuildFlags.MEMORY) - value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") + value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") if 'Debug' in value: result.append(BuildFlags.DEBUG) elif 'RelWithDebInfo' in value or 'Release' in value: result.append(BuildFlags.RELEASE) - value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") + value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") if value in ('ON', '1'): result.append(BuildFlags.UNBUNDLED) - value = clickhouse_client.execute_one("SELECT value FROM system.settings WHERE name = 'default_database_engine'") + value = clickhouse_execute_one(args, "SELECT value FROM system.settings WHERE name = 'default_database_engine'") if value == 'Ordinary': result.append(BuildFlags.ORDINARY_DATABASE) - value = int(clickhouse_client.execute_one("SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")) + value = int(clickhouse_execute_one(args, "SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")) if value == 0: result.append(BuildFlags.POLYMORPHIC_PARTS) @@ -1118,9 +1121,8 @@ def main(args): global exit_code global server_logs_level global restarted_tests - global clickhouse_client - if not check_server_started(args.server_check_retries): + if not check_server_started(args): msg = "Server is not responding. Cannot execute 'SELECT 1' query. \ If you are using split build, you have to specify -c option." if args.hung_check: @@ -1130,7 +1132,7 @@ def main(args): print_stacktraces() raise Exception(msg) - args.build_flags = collect_build_flags() + args.build_flags = collect_build_flags(args) if args.skip: args.skip = set(args.skip) @@ -1167,7 +1169,7 @@ def main(args): while create_database_retries < MAX_RETRIES: start_time = datetime.now() try: - clickhouse_client.execute("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name)) + clickhouse_execute(args, "CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name)) except Exception as e: total_time = (datetime.now() - start_time).total_seconds() if not need_retry_error(args, e, total_time): @@ -1384,9 +1386,10 @@ if __name__ == '__main__': tcp_host = os.getenv("CLICKHOUSE_HOST") if tcp_host is not None: + args.tcp_host = tcp_host args.client += f' --host={tcp_host}' else: - tcp_host = 'localhost' + args.tcp_host = 'localhost' tcp_port = os.getenv("CLICKHOUSE_PORT_TCP") if tcp_port is not None: @@ -1398,8 +1401,9 @@ if __name__ == '__main__': client_database = os.getenv("CLICKHOUSE_DATABASE") if client_database is not None: args.client += f' --database={client_database}' + args.client_database = client_database else: - client_database = 'default' + args.client_database = 'default' if args.client_option: # Set options for client @@ -1431,9 +1435,4 @@ if __name__ == '__main__': pandas.options.display.max_columns = None pandas.options.display.width = None - clickhouse_client = Client(host=tcp_host, - port=args.tcp_port, - database=client_database, - settings=get_additional_client_options_dict(args)) - main(args) From f854065744f57607e23a0de3edcca1b06f06c11a Mon Sep 17 00:00:00 2001 From: Alexey Date: Fri, 8 Oct 2021 05:05:12 +0000 Subject: [PATCH 291/950] buildID() description --- docs/en/sql-reference/functions/other-functions.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index afcc9563b58..9828c91909b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -699,6 +699,12 @@ If it is executed in the context of a distributed table, then it generates a nor Returns the version of the server as a string. If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +## buildId() {#buildid} + +Returns the compiler build id of the running binary. +If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. + + ## blockNumber {#blocknumber} Returns the sequence number of the data block where the row is located. From d454a9affe73ee5844f19f8e85a4143c89c1d016 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 8 Oct 2021 10:07:55 +0300 Subject: [PATCH 292/950] Update docs/ru/getting-started/example-datasets/metrica.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/metrica.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index 27105ca8488..4d862bae423 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -35,7 +35,7 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` bash $ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv -$ # теперь создадим таблицу +# создадим таблицу hits_v1 $ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" $ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # for hits_100m_obfuscated From 9d97a1263f4bd31c8520d9a54a35c8aa8877b982 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 8 Oct 2021 10:08:02 +0300 Subject: [PATCH 293/950] Update docs/ru/getting-started/example-datasets/metrica.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/metrica.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index 4d862bae423..765642ce3ae 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -38,7 +38,7 @@ $ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads= # создадим таблицу hits_v1 $ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" $ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" -# for hits_100m_obfuscated +# создадим таблицу hits_100m_obfuscated clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" $ # импортируем данные From 2c5341df33a3410db2aed4f57d4429d681064186 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 8 Oct 2021 10:08:25 +0300 Subject: [PATCH 294/950] Update docs/ru/sql-reference/functions/geo/h3.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/geo/h3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md index e85236848f6..db96f0caa1d 100644 --- a/docs/ru/sql-reference/functions/geo/h3.md +++ b/docs/ru/sql-reference/functions/geo/h3.md @@ -6,7 +6,7 @@ toc_title: "Функции для работы с индексами H3" [H3](https://eng.uber.com/h3/) — это система геокодирования, которая делит поверхность Земли на равные шестигранные ячейки. Система поддерживает иерархию (вложенность) ячеек, т.е. каждый "родительский" шестигранник может быть поделен на семь одинаковых вложенных "дочерних" шестигранников, и так далее. -Уровень вложенности называется `разрешением` и может принимать значение от `0` до `15`, где `0` соответствует `базовым` ячейкам самого верхнего уровня (наиболее крупным). +Уровень вложенности называется "разрешением" и может принимать значение от `0` до `15`, где `0` соответствует "базовым" ячейкам самого верхнего уровня (наиболее крупным). Для каждой точки, имеющей широту и долготу, можно получить 64-битный индекс H3, соответствующий номеру шестигранной ячейки, где эта точка находится. From 0fdbf867a45689308f0072a005a70b83757d8cc0 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 8 Oct 2021 10:08:31 +0300 Subject: [PATCH 295/950] Update docs/ru/getting-started/example-datasets/metrica.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/metrica.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index 765642ce3ae..761e298fc54 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -41,9 +41,9 @@ $ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, Ja # создадим таблицу hits_100m_obfuscated clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" -$ # импортируем данные +# импортируем данные $ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 -$ # опционально можно оптимизировать таблицу +# опционально можно оптимизировать таблицу $ clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL" $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` From d11cae26178fd114f69c763de4dbd18190175486 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 8 Oct 2021 10:26:02 +0300 Subject: [PATCH 296/950] Remove `$ #` from metrica.md --- docs/ru/getting-started/example-datasets/metrica.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index 761e298fc54..0f5e7197fe5 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -14,7 +14,7 @@ toc_title: "Анонимизированные данные Яндекс.Мет ``` bash $ curl -O https://datasets.clickhouse.com/hits/partitions/hits_v1.tar $ tar xvf hits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse -$ # убедитесь, что установлены корректные права доступа на файлы +# убедитесь, что установлены корректные права доступа на файлы $ sudo service clickhouse-server restart $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` @@ -24,7 +24,7 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` bash $ curl -O https://datasets.clickhouse.com/visits/partitions/visits_v1.tar $ tar xvf visits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse -$ # убедитесь, что установлены корректные права доступа на файлы +# убедитесь, что установлены корректные права доступа на файлы $ sudo service clickhouse-server restart $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` @@ -52,12 +52,12 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` bash $ curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv -$ # теперь создадим таблицу +# теперь создадим таблицу $ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" $ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" -$ # импортируем данные +# импортируем данные $ cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 -$ # опционально можно оптимизировать таблицу +# опционально можно оптимизировать таблицу $ clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL" $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` From 383f3a3a20ff7268b4c2fad14d0e20f266682b77 Mon Sep 17 00:00:00 2001 From: vesslanjin Date: Fri, 8 Oct 2021 09:19:58 -0400 Subject: [PATCH 297/950] =?UTF-8?q?Remove=C2=A0branchy=C2=A0code=C2=A0in?= =?UTF-8?q?=C2=A0filter=C2=A0operation=C2=A0with=C2=A0a=C2=A0better=C2=A0i?= =?UTF-8?q?mplementation=C2=A0with=C2=A0popcnt/ctz=C2=A0which=C2=A0have?= =?UTF-8?q?=C2=A0better=C2=A0performance?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Zhu Jasper --- src/Columns/ColumnFixedString.cpp | 22 +++++++++------------- src/Columns/ColumnVector.cpp | 11 +++++++---- src/Columns/ColumnsCommon.cpp | 11 +++++++---- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index ce39ab0994c..4bfc6513263 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -231,7 +231,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result const UInt8 * filt_end = filt_pos + col_size; const UInt8 * data_pos = chars.data(); -#ifdef __SSE2__ +#if defined(__SSE2__) && defined(__POPCNT__) /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -251,28 +251,24 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result if (0 == mask) { /// Nothing is inserted. - data_pos += chars_per_simd_elements; } else if (0xFFFF == mask) { res->chars.insert(data_pos, data_pos + chars_per_simd_elements); - data_pos += chars_per_simd_elements; } else { size_t res_chars_size = res->chars.size(); - for (size_t i = 0; i < SIMD_BYTES; ++i) - { - if (filt_pos[i]) - { - res->chars.resize(res_chars_size + n); - memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos, n); - res_chars_size += n; - } - data_pos += n; + size_t pcnt = __builtin_popcount(mask); + for(size_t j = 0; j < pcnt; j++) { + size_t index = __builtin_ctz(mask); + res->chars.resize(res_chars_size + n); + memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos+index*n, n); + res_chars_size += n; + mask = mask & (mask-1); } } - + data_pos += chars_per_simd_elements; filt_pos += SIMD_BYTES; } #endif diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 7f3cdaeec7f..ff84204a7cb 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -311,7 +311,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); -#ifdef __SSE2__ +#if defined(__SSE2__) && defined(__POPCNT__) /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -337,9 +337,12 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s } else { - for (size_t i = 0; i < SIMD_BYTES; ++i) - if (filt_pos[i]) - res_data.push_back(data_pos[i]); + size_t pcnt = __builtin_popcount(mask); + for(size_t j = 0; j < pcnt; j++) { + size_t index = __builtin_ctz(mask); + res_data.push_back(data_pos[index]); + mask = mask & (mask-1); + } } filt_pos += SIMD_BYTES; diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 41933ed08ed..5c0214054b2 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -229,7 +229,7 @@ namespace memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T)); }; - #ifdef __SSE2__ + #if defined(__SSE2__) && defined(__POPCNT__) const __m128i zero_vec = _mm_setzero_si128(); static constexpr size_t SIMD_BYTES = 16; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; @@ -262,9 +262,12 @@ namespace } else { - for (size_t i = 0; i < SIMD_BYTES; ++i) - if (filt_pos[i]) - copy_array(offsets_pos + i); + size_t pcnt = __builtin_popcount(mask); + for(size_t j = 0; j < pcnt; j++) { + size_t index = __builtin_ctz(mask); + copy_array(offsets_pos + index); + mask = mask & (mask-1); + } } filt_pos += SIMD_BYTES; From 35feb082365eb0517f2c58659507de34ed47d3af Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 8 Oct 2021 12:03:55 +0300 Subject: [PATCH 298/950] Whitespace changes --- src/Columns/ColumnFixedString.cpp | 7 ++++--- src/Columns/ColumnVector.cpp | 5 +++-- src/Columns/ColumnsCommon.cpp | 5 +++-- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 4bfc6513263..1080f29b5aa 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -260,12 +260,13 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result { size_t res_chars_size = res->chars.size(); size_t pcnt = __builtin_popcount(mask); - for(size_t j = 0; j < pcnt; j++) { + for (size_t j = 0; j < pcnt; ++j) + { size_t index = __builtin_ctz(mask); res->chars.resize(res_chars_size + n); - memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos+index*n, n); + memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); res_chars_size += n; - mask = mask & (mask-1); + mask = mask & (mask - 1); } } data_pos += chars_per_simd_elements; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index ff84204a7cb..e7c56a57d51 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -338,10 +338,11 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s else { size_t pcnt = __builtin_popcount(mask); - for(size_t j = 0; j < pcnt; j++) { + for (size_t j = 0; j < pcnt; ++j) + { size_t index = __builtin_ctz(mask); res_data.push_back(data_pos[index]); - mask = mask & (mask-1); + mask = mask & (mask - 1); } } diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 5c0214054b2..b4614abe490 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -263,10 +263,11 @@ namespace else { size_t pcnt = __builtin_popcount(mask); - for(size_t j = 0; j < pcnt; j++) { + for (size_t j = 0; j < pcnt; ++j) + { size_t index = __builtin_ctz(mask); copy_array(offsets_pos + index); - mask = mask & (mask-1); + mask = mask & (mask - 1); } } From a4d1ad61d0e220b6c215eaffd4cf3531e4c01423 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 8 Oct 2021 12:09:12 +0300 Subject: [PATCH 299/950] Remove $ symbols in bash commands. --- .../example-datasets/metrica.md | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index 0f5e7197fe5..15c0cc14517 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -12,21 +12,21 @@ toc_title: "Анонимизированные данные Яндекс.Мет **Скачивание и импортирование партиций hits:** ``` bash -$ curl -O https://datasets.clickhouse.com/hits/partitions/hits_v1.tar -$ tar xvf hits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse +curl -O https://datasets.clickhouse.com/hits/partitions/hits_v1.tar +tar xvf hits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse # убедитесь, что установлены корректные права доступа на файлы -$ sudo service clickhouse-server restart -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" +sudo service clickhouse-server restart +clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` **Скачивание и импортирование партиций visits:** ``` bash -$ curl -O https://datasets.clickhouse.com/visits/partitions/visits_v1.tar -$ tar xvf visits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse +curl -O https://datasets.clickhouse.com/visits/partitions/visits_v1.tar +tar xvf visits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse # убедитесь, что установлены корректные права доступа на файлы -$ sudo service clickhouse-server restart -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" +sudo service clickhouse-server restart +clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` ## Получение таблиц из сжатых tsv-файлов {#poluchenie-tablits-iz-szhatykh-tsv-failov} @@ -34,32 +34,32 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" **Скачивание и импортирование hits из сжатого tsv-файла** ``` bash -$ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv +curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv # создадим таблицу hits_v1 -$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -$ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" +clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # создадим таблицу hits_100m_obfuscated clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # импортируем данные -$ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 +cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 # опционально можно оптимизировать таблицу -$ clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL" -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" +clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL" +clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` **Скачивание и импортирование visits из сжатого tsv-файла** ``` bash -$ curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv +curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv # теперь создадим таблицу -$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -$ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" +clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # импортируем данные -$ cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 +cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 # опционально можно оптимизировать таблицу -$ clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL" -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" +clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL" +clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` ## Запросы {#zaprosy} From 525df998934368aedb3baac72645453b080cf01d Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 8 Oct 2021 12:10:19 +0300 Subject: [PATCH 300/950] Whitespace fix --- src/Columns/ColumnVector.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index e7c56a57d51..61ba2074bd7 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -343,7 +343,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s size_t index = __builtin_ctz(mask); res_data.push_back(data_pos[index]); mask = mask & (mask - 1); - } + } } filt_pos += SIMD_BYTES; From 757f49bacbd9ced52d3f0b333601fbe6f75e6ead Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 8 Oct 2021 11:15:47 +0000 Subject: [PATCH 301/950] test poco --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..7f5c60d2b0c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,6 +1,6 @@ [submodule "contrib/poco"] path = contrib/poco - url = https://github.com/ClickHouse-Extras/poco.git + url = https://github.com/ucasfl/poco.git branch = clickhouse [submodule "contrib/zstd"] path = contrib/zstd From 3acdcc6ec5b31d3b5e7c33e0e49df99b39dbdda4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 15:03:31 +0300 Subject: [PATCH 302/950] Update example --- src/Core/examples/coro.cpp | 44 ++++++++++++++------------------------ 1 file changed, 16 insertions(+), 28 deletions(-) diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp index c8e2f7418e4..a9728314495 100644 --- a/src/Core/examples/coro.cpp +++ b/src/Core/examples/coro.cpp @@ -12,37 +12,25 @@ #include #if defined(__clang__) - #include -template -using coroutine_handle = std::experimental::coroutine_handle; - -using default_coroutine_handle = std::experimental::coroutine_handle<>; - -using suspend_never = std::experimental::suspend_never; -using suspend_always = std::experimental::suspend_always; +namespace std +{ + using namespace experimental::coroutines_v1; +} #else - #include - -template -using coroutine_handle = std::coroutine_handle; - -using default_coroutine_handle = std::coroutine_handle<>; - -using suspend_never = std::suspend_never; -using suspend_always = std::suspend_always; - +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wzero-as-null-pointer-constant" #endif template -struct suspend_never_val +struct suspend_value { constexpr bool await_ready() const noexcept { return true; } - constexpr void await_suspend(default_coroutine_handle) const noexcept {} + constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} constexpr T await_resume() const noexcept { std::cout << " ret " << val << std::endl; @@ -57,10 +45,10 @@ struct resumable { struct promise_type { - using coro_handle = coroutine_handle; + using coro_handle = std::coroutine_handle; auto get_return_object() { return coro_handle::from_promise(*this); } - auto initial_suspend() { return suspend_never(); } - auto final_suspend() noexcept { return suspend_never_val{*r->value}; } + auto initial_suspend() { return std::suspend_never(); } + auto final_suspend() noexcept { return suspend_value{*r->value}; } //void return_void() {} void return_value(T value_) { r->value = value_; } void unhandled_exception() @@ -76,7 +64,7 @@ struct resumable resumable * r = nullptr; }; - using coro_handle = coroutine_handle; + using coro_handle = std::coroutine_handle; bool await_ready() const noexcept { return false; } void await_suspend(coro_handle g) noexcept @@ -148,16 +136,16 @@ private: std::exception_ptr exception; }; -resumable boo(std::string tag) +resumable boo([[maybe_unused]] std::string tag) { std::cout << "x" << std::endl; - co_await suspend_always(); + co_await std::suspend_always(); std::cout << StackTrace().toString(); std::cout << "y" << std::endl; co_return 1; } -resumable bar(std::string tag) +resumable bar([[maybe_unused]] std::string tag) { std::cout << "a" << std::endl; int res1 = co_await boo("boo1"); @@ -169,7 +157,7 @@ resumable bar(std::string tag) co_return res1 + res2; // 1 + 1 = 2 } -resumable foo(std::string tag) { +resumable foo([[maybe_unused]] std::string tag) { std::cout << "Hello" << std::endl; auto res1 = co_await bar("bar1"); std::cout << "Coro " << res1 << std::endl; From b8e77a0c62ca664e65f545a39c1132edb14995e1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 15:06:09 +0300 Subject: [PATCH 303/950] Add -fcoroutines for g++ --- CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index 66df4d3124a..685b2c25a0d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -336,6 +336,10 @@ if (COMPILER_GCC OR COMPILER_CLANG) set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") endif () +if (COMPILER_GCC) + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fcoroutines") +endif () + # Compiler-specific coverage flags e.g. -fcoverage-mapping for gcc option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF) From 7fa3257bb6b5bf5ef8576f70c4657335442a0c70 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Fri, 8 Oct 2021 15:37:24 +0300 Subject: [PATCH 304/950] correct test --- tests/integration/test_settings_profile/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 7be0b395764..78efe7ffcad 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -201,7 +201,7 @@ def test_show_profiles(): assert instance.query("SHOW CREATE PROFILE xyz") == "CREATE SETTINGS PROFILE xyz\n" assert instance.query( - "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" + "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" assert instance.query( "SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" \ "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" \ From 340b53ef853348758c9042b16a8599120ebc8d22 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 17:03:54 +0300 Subject: [PATCH 305/950] Remove some more streams. --- programs/benchmark/Benchmark.cpp | 17 ++-- programs/copier/ClusterCopier.cpp | 1 - programs/copier/Internals.h | 2 +- src/Client/ClientBase.cpp | 11 ++- src/Client/ClientBase.h | 7 +- src/Columns/ColumnAggregateFunction.cpp | 2 +- src/Columns/ColumnArray.cpp | 2 +- src/Columns/ColumnDecimal.cpp | 2 +- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnLowCardinality.cpp | 2 +- src/Columns/ColumnMap.cpp | 2 +- src/Columns/ColumnNullable.cpp | 2 +- src/Columns/ColumnString.cpp | 2 +- src/Columns/ColumnTuple.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- .../ExpressionBlockInputStream.cpp | 39 ---------- src/DataStreams/ExpressionBlockInputStream.h | 52 ------------- ...wOutputStream.cpp => InternalTextLogs.cpp} | 9 +-- ...gsRowOutputStream.h => InternalTextLogs.h} | 9 +-- .../MaterializingBlockInputStream.cpp | 28 ------- .../MaterializingBlockInputStream.h | 21 ----- src/DataStreams/RemoteBlockInputStream.cpp | 69 ---------------- src/DataStreams/RemoteBlockInputStream.h | 78 ------------------- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- .../PostgreSQLDictionarySource.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 5 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/ThreadStatusExt.cpp | 2 +- .../getHeaderForProcessingStage.cpp | 8 +- .../Algorithms/CollapsingSortedAlgorithm.h | 2 +- .../Algorithms/MergingSortedAlgorithm.cpp | 2 +- .../Algorithms/ReplacingSortedAlgorithm.h | 2 +- .../Algorithms/VersionedCollapsingAlgorithm.h | 2 +- .../Merges/MergingSortedTransform.cpp | 2 +- .../Sources/SourceFromInputStream.cpp | 5 -- .../Transforms/ColumnGathererTransform.cpp} | 2 +- .../Transforms/ColumnGathererTransform.h} | 0 .../Transforms/DistinctSortedTransform.cpp} | 2 +- .../Transforms/DistinctSortedTransform.h} | 0 .../Transforms}/MongoDBSource.cpp | 0 .../Transforms}/MongoDBSource.h | 0 .../Transforms}/PostgreSQLSource.cpp | 0 .../Transforms}/PostgreSQLSource.h | 0 .../Transforms/buildPushingToViewsChain.cpp} | 2 +- .../Transforms/buildPushingToViewsChain.h} | 0 src/Storages/MergeTree/MergeTask.cpp | 5 +- src/Storages/MergeTree/MergeTask.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +- src/Storages/MergeTree/MutateTask.cpp | 6 +- .../PostgreSQLReplicationHandler.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 12 +-- .../tests/gtest_row_source_bits_test.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 2 +- 58 files changed, 73 insertions(+), 378 deletions(-) delete mode 100644 src/DataStreams/ExpressionBlockInputStream.cpp delete mode 100644 src/DataStreams/ExpressionBlockInputStream.h rename src/DataStreams/{InternalTextLogsRowOutputStream.cpp => InternalTextLogs.cpp} (94%) rename src/DataStreams/{InternalTextLogsRowOutputStream.h => InternalTextLogs.h} (57%) delete mode 100644 src/DataStreams/MaterializingBlockInputStream.cpp delete mode 100644 src/DataStreams/MaterializingBlockInputStream.h delete mode 100644 src/DataStreams/RemoteBlockInputStream.cpp delete mode 100644 src/DataStreams/RemoteBlockInputStream.h rename src/{DataStreams/ColumnGathererStream.cpp => Processors/Transforms/ColumnGathererTransform.cpp} (98%) rename src/{DataStreams/ColumnGathererStream.h => Processors/Transforms/ColumnGathererTransform.h} (100%) rename src/{DataStreams/DistinctSortedBlockInputStream.cpp => Processors/Transforms/DistinctSortedTransform.cpp} (98%) rename src/{DataStreams/DistinctSortedBlockInputStream.h => Processors/Transforms/DistinctSortedTransform.h} (100%) rename src/{DataStreams => Processors/Transforms}/MongoDBSource.cpp (100%) rename src/{DataStreams => Processors/Transforms}/MongoDBSource.h (100%) rename src/{DataStreams => Processors/Transforms}/PostgreSQLSource.cpp (100%) rename src/{DataStreams => Processors/Transforms}/PostgreSQLSource.h (100%) rename src/{DataStreams/PushingToViewsBlockOutputStream.cpp => Processors/Transforms/buildPushingToViewsChain.cpp} (99%) rename src/{DataStreams/PushingToViewsBlockOutputStream.h => Processors/Transforms/buildPushingToViewsChain.h} (100%) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index be57a3b92a0..caa0a87bde2 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include #include #include @@ -424,20 +424,19 @@ private: if (reconnect) connection.disconnect(); - RemoteBlockInputStream stream( + RemoteQueryExecutor executor( connection, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage); if (!query_id.empty()) - stream.setQueryId(query_id); + executor.setQueryId(query_id); Progress progress; - stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); + executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); - stream.readPrefix(); - while (Block block = stream.read()); + BlockStreamProfileInfo info; + while (Block block = executor.read()) + info.update(block); - stream.readSuffix(); - - const BlockStreamProfileInfo & info = stream.getProfileInfo(); + executor.finish(); double seconds = watch.elapsedSeconds(); diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 30b99b69351..60e1590d3bd 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -14,7 +14,6 @@ #include #include #include -#include namespace DB { diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 54e7e7719e0..45531e750b9 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -49,7 +49,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index cde5a5f9977..99cadabfaea 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -48,7 +48,7 @@ #include #include -#include +#include namespace fs = std::filesystem; @@ -95,6 +95,9 @@ void interruptSignalHandler(int signum) _exit(signum); } +ClientBase::~ClientBase() = default; +ClientBase::ClientBase() = default; + void ClientBase::setupSignalHandler() { exit_on_signal.test_and_set(); @@ -393,8 +396,7 @@ void ClientBase::initLogsOutputStream() } } - logs_out_stream = std::make_shared(*wb, stdout_is_a_tty); - logs_out_stream->writePrefix(); + logs_out_stream = std::make_unique(*wb, stdout_is_a_tty); } } @@ -641,9 +643,6 @@ void ClientBase::onEndOfStream() if (block_out_stream) block_out_stream->writeSuffix(); - if (logs_out_stream) - logs_out_stream->writeSuffix(); - resetOutput(); if (is_interactive && !written_first_block) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index bf9e8fdfe47..b122803e1db 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -32,12 +32,17 @@ enum MultiQueryProcessingStage void interruptSignalHandler(int signum); +class InternalTextLogs; + class ClientBase : public Poco::Util::Application { public: using Arguments = std::vector; + ClientBase(); + ~ClientBase() override; + void init(int argc, char ** argv); protected: @@ -177,7 +182,7 @@ protected: /// The user could specify special file for server logs (stderr by default) std::unique_ptr out_logs_buf; String server_logs_file; - BlockOutputStreamPtr logs_out_stream; + std::unique_ptr logs_out_stream; String home_path; String history_file; /// Path to a file containing command history. diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 5224ff0fbdb..cd1185d99db 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 8e613ae93c2..5e40b89cc7e 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -13,7 +13,7 @@ #include #include -#include +#include #include #include diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index d50138bc582..d8c5ced4b6b 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale); diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index ce39ab0994c..2bfd46a9607 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 6f7bc58406d..5b33baab345 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 03addee4f8d..4374ca30c99 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index d4d62f9580c..4b3340a6b50 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index bb34f524fdf..2beb9add318 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index f053c32fda1..d157f18bf32 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 7f3cdaeec7f..56c2cef3243 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/DataStreams/ExpressionBlockInputStream.cpp b/src/DataStreams/ExpressionBlockInputStream.cpp deleted file mode 100644 index 4840a6263f6..00000000000 --- a/src/DataStreams/ExpressionBlockInputStream.cpp +++ /dev/null @@ -1,39 +0,0 @@ -#include -#include - - -namespace DB -{ - -ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_) - : expression(expression_) -{ - children.push_back(input); - cached_header = children.back()->getHeader(); - expression->execute(cached_header, true); -} - -String ExpressionBlockInputStream::getName() const { return "Expression"; } - -Block ExpressionBlockInputStream::getTotals() -{ - totals = children.back()->getTotals(); - expression->execute(totals); - - return totals; -} - -Block ExpressionBlockInputStream::getHeader() const -{ - return cached_header.cloneEmpty(); -} - -Block ExpressionBlockInputStream::readImpl() -{ - Block res = children.back()->read(); - if (res) - expression->execute(res); - return res; -} - -} diff --git a/src/DataStreams/ExpressionBlockInputStream.h b/src/DataStreams/ExpressionBlockInputStream.h deleted file mode 100644 index fae54fbcfbf..00000000000 --- a/src/DataStreams/ExpressionBlockInputStream.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -class ExpressionActions; - -/** Executes a certain expression over the block. - * The expression consists of column identifiers from the block, constants, common functions. - * For example: hits * 2 + 3, url LIKE '%yandex%' - * The expression processes each row independently of the others. - */ -class ExpressionBlockInputStream : public IBlockInputStream -{ -public: - using ExpressionActionsPtr = std::shared_ptr; - - ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_); - - String getName() const override; - Block getTotals() override; - Block getHeader() const override; - -protected: - ExpressionActionsPtr expression; - - Block readImpl() override; - -private: - Block cached_header; -}; - -/// ExpressionBlockInputStream that could generate many out blocks for single input block. -class InflatingExpressionBlockInputStream : public ExpressionBlockInputStream -{ -public: - InflatingExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_) - : ExpressionBlockInputStream(input, expression_) - {} - -protected: - Block readImpl() override; - -private: - ExtraBlockPtr not_processed; - size_t action_number = 0; -}; - -} diff --git a/src/DataStreams/InternalTextLogsRowOutputStream.cpp b/src/DataStreams/InternalTextLogs.cpp similarity index 94% rename from src/DataStreams/InternalTextLogsRowOutputStream.cpp rename to src/DataStreams/InternalTextLogs.cpp index 14247b8f2aa..a5883d17f28 100644 --- a/src/DataStreams/InternalTextLogsRowOutputStream.cpp +++ b/src/DataStreams/InternalTextLogs.cpp @@ -1,4 +1,4 @@ -#include "InternalTextLogsRowOutputStream.h" +#include "InternalTextLogs.h" #include #include #include @@ -13,12 +13,7 @@ namespace DB { -Block InternalTextLogsRowOutputStream::getHeader() const -{ - return InternalTextLogsQueue::getSampleBlock(); -} - -void InternalTextLogsRowOutputStream::write(const Block & block) +void InternalTextLogs::write(const Block & block) { const auto & array_event_time = typeid_cast(*block.getByName("event_time").column).getData(); const auto & array_microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData(); diff --git a/src/DataStreams/InternalTextLogsRowOutputStream.h b/src/DataStreams/InternalTextLogs.h similarity index 57% rename from src/DataStreams/InternalTextLogsRowOutputStream.h rename to src/DataStreams/InternalTextLogs.h index 8ade76b34a7..1312c1d327c 100644 --- a/src/DataStreams/InternalTextLogsRowOutputStream.h +++ b/src/DataStreams/InternalTextLogs.h @@ -9,16 +9,15 @@ namespace DB /// Prints internal server logs /// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock() /// NOTE: IRowOutputFormat does not suite well for this case -class InternalTextLogsRowOutputStream : public IBlockOutputStream +class InternalTextLogs { public: - InternalTextLogsRowOutputStream(WriteBuffer & buf_out, bool color_) : wb(buf_out), color(color_) {} + InternalTextLogs(WriteBuffer & buf_out, bool color_) : wb(buf_out), color(color_) {} - Block getHeader() const override; - void write(const Block & block) override; + void write(const Block & block); - void flush() override + void flush() { wb.next(); } diff --git a/src/DataStreams/MaterializingBlockInputStream.cpp b/src/DataStreams/MaterializingBlockInputStream.cpp deleted file mode 100644 index e0f287d44fa..00000000000 --- a/src/DataStreams/MaterializingBlockInputStream.cpp +++ /dev/null @@ -1,28 +0,0 @@ -#include -#include - - -namespace DB -{ - -MaterializingBlockInputStream::MaterializingBlockInputStream(const BlockInputStreamPtr & input) -{ - children.push_back(input); -} - -String MaterializingBlockInputStream::getName() const -{ - return "Materializing"; -} - -Block MaterializingBlockInputStream::getHeader() const -{ - return materializeBlock(children.back()->getHeader()); -} - -Block MaterializingBlockInputStream::readImpl() -{ - return materializeBlock(children.back()->read()); -} - -} diff --git a/src/DataStreams/MaterializingBlockInputStream.h b/src/DataStreams/MaterializingBlockInputStream.h deleted file mode 100644 index 90fd91bd550..00000000000 --- a/src/DataStreams/MaterializingBlockInputStream.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -/** Converts columns-constants to full columns ("materializes" them). - */ -class MaterializingBlockInputStream : public IBlockInputStream -{ -public: - MaterializingBlockInputStream(const BlockInputStreamPtr & input); - String getName() const override; - Block getHeader() const override; - -protected: - Block readImpl() override; -}; - -} diff --git a/src/DataStreams/RemoteBlockInputStream.cpp b/src/DataStreams/RemoteBlockInputStream.cpp deleted file mode 100644 index 7caa54cff22..00000000000 --- a/src/DataStreams/RemoteBlockInputStream.cpp +++ /dev/null @@ -1,69 +0,0 @@ -#include -#include - -namespace DB -{ - -RemoteBlockInputStream::RemoteBlockInputStream( - Connection & connection, - const String & query_, const Block & header_, ContextPtr context_, - const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) - : query_executor(connection, query_, header_, context_, throttler, scalars_, external_tables_, stage_) -{ - init(); -} - -RemoteBlockInputStream::RemoteBlockInputStream( - const ConnectionPoolWithFailoverPtr & pool, - std::vector && connections, - const String & query_, const Block & header_, ContextPtr context_, - const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) - : query_executor(pool, std::move(connections), query_, header_, context_, throttler, scalars_, external_tables_, stage_) -{ - init(); -} - -RemoteBlockInputStream::RemoteBlockInputStream( - const ConnectionPoolWithFailoverPtr & pool, - const String & query_, const Block & header_, ContextPtr context_, - const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) - : query_executor(pool, query_, header_, context_, throttler, scalars_, external_tables_, stage_) -{ - init(); -} - -void RemoteBlockInputStream::init() -{ - query_executor.setProgressCallback([this](const Progress & progress) { progressImpl(progress); }); - query_executor.setProfileInfoCallback([this](const BlockStreamProfileInfo & info_) { info.setFrom(info_, true); }); - query_executor.setLogger(log); -} - -void RemoteBlockInputStream::cancel(bool kill) -{ - if (kill) - is_killed = true; - - bool old_val = false; - if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) - return; - - query_executor.cancel(); -} - -Block RemoteBlockInputStream::readImpl() -{ - auto block = query_executor.read(); - - if (isCancelledOrThrowIfKilled()) - return Block(); - - return block; -} - -void RemoteBlockInputStream::readSuffixImpl() -{ - query_executor.finish(); -} - -} diff --git a/src/DataStreams/RemoteBlockInputStream.h b/src/DataStreams/RemoteBlockInputStream.h deleted file mode 100644 index 60bfa5d6990..00000000000 --- a/src/DataStreams/RemoteBlockInputStream.h +++ /dev/null @@ -1,78 +0,0 @@ -#pragma once - -#include - -#include - -#include -#include -#include -#include -#include - -#include - -namespace DB -{ - -class Context; - -/** This class allows one to launch queries on remote replicas of one shard and get results - */ -class RemoteBlockInputStream : public IBlockInputStream -{ -public: - /// Takes already set connection. - RemoteBlockInputStream( - Connection & connection, - const String & query_, const Block & header_, ContextPtr context_, - const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); - - /// Accepts several connections already taken from pool. - RemoteBlockInputStream( - const ConnectionPoolWithFailoverPtr & pool, - std::vector && connections, - const String & query_, const Block & header_, ContextPtr context_, - const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); - - /// Takes a pool and gets one or several connections from it. - RemoteBlockInputStream( - const ConnectionPoolWithFailoverPtr & pool, - const String & query_, const Block & header_, ContextPtr context_, - const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); - - /// Set the query_id. For now, used by performance test to later find the query - /// in the server query_log. Must be called before sending the query to the server. - void setQueryId(const std::string & query_id) { query_executor.setQueryId(query_id); } - - /// Specify how we allocate connections on a shard. - void setPoolMode(PoolMode pool_mode) { query_executor.setPoolMode(pool_mode); } - - void setMainTable(StorageID main_table_) { query_executor.setMainTable(std::move(main_table_)); } - - /// Prevent default progress notification because progress' callback is called by its own. - void progress(const Progress & /*value*/) override {} - - void cancel(bool kill) override; - - String getName() const override { return "Remote"; } - - Block getHeader() const override { return query_executor.getHeader(); } - Block getTotals() override { return query_executor.getTotals(); } - Block getExtremes() override { return query_executor.getExtremes(); } - -protected: - Block readImpl() override; - void readSuffixImpl() override; - -private: - RemoteQueryExecutor query_executor; - Poco::Logger * log = &Poco::Logger::get("RemoteBlockInputStream"); - - void init(); -}; - -} diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 21c69b71b67..1e8be726941 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -67,7 +67,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) // Poco/MongoDB/BSONWriter.h:54: void writeCString(const std::string & value); // src/IO/WriteHelpers.h:146 #define writeCString(s, buf) #include -#include +#include namespace DB diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 484d16e1f03..a3324b7d769 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -8,7 +8,7 @@ #if USE_LIBPQXX #include #include -#include +#include #include "readInvalidateQuery.h" #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 41b724c690e..a36941ea07a 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 12922e45a55..85cc889319f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -65,6 +65,7 @@ #include #include #include +#include #include #include #include @@ -1815,8 +1816,8 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc Block block_with_count{ {std::move(column), std::make_shared(func, argument_types, desc.parameters), desc.column_name}}; - auto istream = std::make_shared(block_with_count); - auto prepared_count = std::make_unique(Pipe(std::make_shared(istream)), context); + auto source = std::make_shared(block_with_count); + auto prepared_count = std::make_unique(Pipe(std::move(source)), context); prepared_count->setStepDescription("Optimized trivial count"); query_plan.addStep(std::move(prepared_count)); from_stage = QueryProcessingStage::WithMergeableState; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e5a129cbe12..2c12c4a6879 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -214,7 +214,7 @@ bool isStorageTouchedByMutations( ASTPtr select_query = prepareQueryAffectedAST(commands, storage, context_copy); /// Interpreter must be alive, when we use result of execute() method. - /// For some reason it may copy context and and give it into ExpressionBlockInputStream + /// For some reason it may copy context and and give it into ExpressionTransform /// after that we will use context from destroyed stack frame in our stream. InterpreterSelectQuery interpreter(select_query, context_copy, storage, metadata_snapshot, SelectQueryOptions().ignoreLimits()); auto io = interpreter.execute(); diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index ddfb832f6b2..465b8e31b08 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 19837cc05d9..1510b477725 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include namespace DB { @@ -120,9 +120,9 @@ Block getHeaderForProcessingStage( TreeRewriterResult new_rewriter_result = *query_info.syntax_analyzer_result; removeJoin(*query->as(), new_rewriter_result, context); - auto stream = std::make_shared( - metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID())); - return InterpreterSelectQuery(query, context, stream, SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); + auto pipe = Pipe(std::make_shared( + metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()))); + return InterpreterSelectQuery(query, context, std::move(pipe), SelectQueryOptions(processed_stage).analyze()).getSampleBlock(); } } throw Exception("Logical Error: unknown processed stage.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 18ebaad5596..f457af05bd5 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace Poco { diff --git a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp index 511bd9dd74f..6b2f0f571a1 100644 --- a/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/MergingSortedAlgorithm.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index 7a193470f89..ca3bbd9d757 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace Poco { diff --git a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h index 6e859d86020..2226762d541 100644 --- a/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/VersionedCollapsingAlgorithm.h @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 62c7f4f762f..667972e3cf6 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/Processors/Sources/SourceFromInputStream.cpp b/src/Processors/Sources/SourceFromInputStream.cpp index 7c88c6dfbeb..57e449370a5 100644 --- a/src/Processors/Sources/SourceFromInputStream.cpp +++ b/src/Processors/Sources/SourceFromInputStream.cpp @@ -1,7 +1,6 @@ #include #include #include -#include namespace DB { @@ -109,10 +108,6 @@ void SourceFromInputStream::work() if (is_stream_finished) return; - /// Don't cancel for RemoteBlockInputStream (otherwise readSuffix can stack) - if (!typeid_cast(stream.get())) - stream->cancel(false); - if (rows_before_limit) { const auto & info = stream->getProfileInfo(); diff --git a/src/DataStreams/ColumnGathererStream.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp similarity index 98% rename from src/DataStreams/ColumnGathererStream.cpp rename to src/Processors/Transforms/ColumnGathererTransform.cpp index 9b2fac79bb0..ddb8a5a0d68 100644 --- a/src/DataStreams/ColumnGathererStream.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/DataStreams/ColumnGathererStream.h b/src/Processors/Transforms/ColumnGathererTransform.h similarity index 100% rename from src/DataStreams/ColumnGathererStream.h rename to src/Processors/Transforms/ColumnGathererTransform.h diff --git a/src/DataStreams/DistinctSortedBlockInputStream.cpp b/src/Processors/Transforms/DistinctSortedTransform.cpp similarity index 98% rename from src/DataStreams/DistinctSortedBlockInputStream.cpp rename to src/Processors/Transforms/DistinctSortedTransform.cpp index 47421941b45..01cef654388 100644 --- a/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/src/Processors/Transforms/DistinctSortedTransform.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/DistinctSortedBlockInputStream.h b/src/Processors/Transforms/DistinctSortedTransform.h similarity index 100% rename from src/DataStreams/DistinctSortedBlockInputStream.h rename to src/Processors/Transforms/DistinctSortedTransform.h diff --git a/src/DataStreams/MongoDBSource.cpp b/src/Processors/Transforms/MongoDBSource.cpp similarity index 100% rename from src/DataStreams/MongoDBSource.cpp rename to src/Processors/Transforms/MongoDBSource.cpp diff --git a/src/DataStreams/MongoDBSource.h b/src/Processors/Transforms/MongoDBSource.h similarity index 100% rename from src/DataStreams/MongoDBSource.h rename to src/Processors/Transforms/MongoDBSource.h diff --git a/src/DataStreams/PostgreSQLSource.cpp b/src/Processors/Transforms/PostgreSQLSource.cpp similarity index 100% rename from src/DataStreams/PostgreSQLSource.cpp rename to src/Processors/Transforms/PostgreSQLSource.cpp diff --git a/src/DataStreams/PostgreSQLSource.h b/src/Processors/Transforms/PostgreSQLSource.h similarity index 100% rename from src/DataStreams/PostgreSQLSource.h rename to src/Processors/Transforms/PostgreSQLSource.h diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp similarity index 99% rename from src/DataStreams/PushingToViewsBlockOutputStream.cpp rename to src/Processors/Transforms/buildPushingToViewsChain.cpp index dac4c0620cb..e5abc3bd416 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/Processors/Transforms/buildPushingToViewsChain.h similarity index 100% rename from src/DataStreams/PushingToViewsBlockOutputStream.h rename to src/Processors/Transforms/buildPushingToViewsChain.h diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 0810d45a805..357659b3bbb 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -20,12 +20,9 @@ #include "Processors/Merges/AggregatingSortedTransform.h" #include "Processors/Merges/VersionedCollapsingTransform.h" #include "Processors/Executors/PipelineExecutingBlockInputStream.h" -#include "DataStreams/DistinctSortedBlockInputStream.h" #include "DataStreams/TTLBlockInputStream.h" #include -#include -#include -#include +#include namespace DB { diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index aceca912cea..05903f94c91 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b35a41d5d19..903f4cd27fc 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -17,11 +17,7 @@ #include #include -#include -#include -#include -#include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e38342e21dd..b8941fc9d84 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,10 +4,8 @@ #include #include #include -#include -#include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 6eb0c568725..873a4b4860c 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,6 +1,6 @@ #include "PostgreSQLReplicationHandler.h" -#include +#include #include #include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 2a1f7cc2aa9..5521b9de39c 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 47a3e4091f5..edb4713a60e 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -1,7 +1,7 @@ #include "StoragePostgreSQL.h" #if USE_LIBPQXX -#include +#include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 50c7fe0610d..416d37cd351 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -60,7 +60,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e4ae27e43bf..c9a0a4873c3 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -11,7 +11,6 @@ #include "Client/Connection.h" #include "Core/QueryProcessingStage.h" #include -#include "DataStreams/RemoteBlockInputStream.h" #include #include #include @@ -32,6 +31,7 @@ #include #include "Processors/Sources/SourceWithProgress.h" #include +#include #include #include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index fb828b8f744..639692beda5 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -71,17 +71,16 @@ ColumnsDescription getStructureOfRemoteTableInShard( }; /// Execute remote query without restrictions (because it's not real user query, but part of implementation) - auto input = std::make_shared(shard_info.pool, query, sample_block, new_context); - input->setPoolMode(PoolMode::GET_ONE); + RemoteQueryExecutor executor(shard_info.pool, query, sample_block, new_context); + executor.setPoolMode(PoolMode::GET_ONE); if (!table_func_ptr) - input->setMainTable(table_id); - input->readPrefix(); + executor.setMainTable(table_id); const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); ParserExpression expr_parser; - while (Block current = input->read()) + while (Block current = executor.read()) { ColumnPtr name = current.getByName("name").column; ColumnPtr type = current.getByName("type").column; @@ -111,6 +110,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( } } + executor.finish(); return res; } diff --git a/src/Storages/tests/gtest_row_source_bits_test.cpp b/src/Storages/tests/gtest_row_source_bits_test.cpp index a6d9179c106..0e7db4b8ab8 100644 --- a/src/Storages/tests/gtest_row_source_bits_test.cpp +++ b/src/Storages/tests/gtest_row_source_bits_test.cpp @@ -1,6 +1,6 @@ #include -#include +#include using DB::RowSourcePart; diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 313ad9d4dcc..160fc3c2468 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include #include From e157d22ef7bb762d7f2f9e88b8aa48447af266b1 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Fri, 8 Oct 2021 18:25:25 +0300 Subject: [PATCH 306/950] Update test.py --- tests/integration/test_settings_profile/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 78efe7ffcad..048d09daaa5 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -203,11 +203,11 @@ def test_show_profiles(): assert instance.query( "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" assert instance.query( - "SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" \ + "SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" \ "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" \ "CREATE SETTINGS PROFILE xyz\n" - expected_access = "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" \ + expected_access = "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" \ "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" \ "CREATE SETTINGS PROFILE xyz\n" assert expected_access in instance.query("SHOW ACCESS") From c6bce1a4cf73c0796050188e3b1bdbc6bc53ce81 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 20:21:19 +0300 Subject: [PATCH 307/950] Update Native. --- src/Client/Connection.cpp | 49 ++++- src/Client/Connection.h | 31 +-- ...eBlockInputStream.cpp => NativeReader.cpp} | 22 +-- ...ativeBlockInputStream.h => NativeReader.h} | 16 +- ...BlockOutputStream.cpp => NativeWriter.cpp} | 8 +- ...tiveBlockOutputStream.h => NativeWriter.h} | 14 +- src/DataStreams/TemporaryFileStream.cpp | 13 +- src/DataStreams/TemporaryFileStream.h | 3 +- src/Formats/FormatFactory.cpp | 4 - src/Formats/NativeFormat.cpp | 81 +++++++- src/Formats/registerFormats.cpp | 4 - src/Interpreters/Aggregator.cpp | 6 +- src/Interpreters/Aggregator.h | 3 +- src/Processors/Formats/Impl/NativeFormat.h | 179 ------------------ .../Sources/SourceFromSingleChunk.cpp | 26 +++ .../Sources/SourceFromSingleChunk.h | 4 +- .../Transforms/AggregatingTransform.cpp | 8 +- .../Transforms/MergeSortingTransform.cpp | 15 +- .../Transforms/SortingTransform.cpp | 4 +- src/Server/TCPHandler.cpp | 12 +- src/Server/TCPHandler.h | 8 +- src/Storages/Distributed/DirectoryMonitor.cpp | 21 +- src/Storages/Distributed/DistributedSink.cpp | 8 +- src/Storages/MergeTree/DataPartsExchange.cpp | 10 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 4 +- .../MergeTree/MergeTreeWriteAheadLog.h | 6 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageSet.cpp | 17 +- src/Storages/StorageStripeLog.cpp | 9 +- utils/wal-dump/main.cpp | 4 +- 30 files changed, 245 insertions(+), 346 deletions(-) rename src/DataStreams/{NativeBlockInputStream.cpp => NativeReader.cpp} (91%) rename src/DataStreams/{NativeBlockInputStream.h => NativeReader.h} (86%) rename src/DataStreams/{NativeBlockOutputStream.cpp => NativeWriter.cpp} (95%) rename src/DataStreams/{NativeBlockOutputStream.h => NativeWriter.h} (78%) delete mode 100644 src/Processors/Formats/Impl/NativeFormat.h create mode 100644 src/Processors/Sources/SourceFromSingleChunk.cpp diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 7324e08bc4c..2aa157bb318 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -58,6 +58,35 @@ namespace ErrorCodes extern const int EMPTY_DATA_PASSED; } +Connection::~Connection() = default; + +Connection::Connection(const String & host_, UInt16 port_, + const String & default_database_, + const String & user_, const String & password_, + const String & cluster_, + const String & cluster_secret_, + const String & client_name_, + Protocol::Compression compression_, + Protocol::Secure secure_, + Poco::Timespan sync_request_timeout_) + : host(host_), port(port_), default_database(default_database_) + , user(user_), password(password_) + , cluster(cluster_) + , cluster_secret(cluster_secret_) + , client_name(client_name_) + , compression(compression_) + , secure(secure_) + , sync_request_timeout(sync_request_timeout_) + , log_wrapper(*this) +{ + /// Don't connect immediately, only on first need. + + if (user.empty()) + user = "default"; + + setDescription(); +} + void Connection::connect(const ConnectionTimeouts & timeouts) { @@ -533,11 +562,11 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) if (!block_out) { if (compression == Protocol::Compression::Enable) - maybe_compressed_out = std::make_shared(*out, compression_codec); + maybe_compressed_out = std::make_unique(*out, compression_codec); else maybe_compressed_out = out; - block_out = std::make_shared(*maybe_compressed_out, server_revision, block.cloneEmpty()); + block_out = std::make_unique(*maybe_compressed_out, server_revision, block.cloneEmpty()); } if (scalar) @@ -866,18 +895,18 @@ Packet Connection::receivePacket() Block Connection::receiveData() { initBlockInput(); - return receiveDataImpl(block_in); + return receiveDataImpl(*block_in); } Block Connection::receiveLogData() { initBlockLogsInput(); - return receiveDataImpl(block_logs_in); + return receiveDataImpl(*block_logs_in); } -Block Connection::receiveDataImpl(BlockInputStreamPtr & stream) +Block Connection::receiveDataImpl(NativeReader & reader) { String external_table_name; readStringBinary(external_table_name, *in); @@ -885,7 +914,7 @@ Block Connection::receiveDataImpl(BlockInputStreamPtr & stream) size_t prev_bytes = in->count(); /// Read one block from network. - Block res = stream->read(); + Block res = reader.read(); if (throttler) throttler->add(in->count() - prev_bytes); @@ -912,7 +941,7 @@ void Connection::initBlockInput() maybe_compressed_in = in; } - block_in = std::make_shared(*maybe_compressed_in, server_revision); + block_in = std::make_unique(*maybe_compressed_in, server_revision); } } @@ -922,7 +951,7 @@ void Connection::initBlockLogsInput() if (!block_logs_in) { /// Have to return superset of SystemLogsQueue::getSampleBlock() columns - block_logs_in = std::make_shared(*in, server_revision); + block_logs_in = std::make_unique(*in, server_revision); } } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 84ce6edac73..a5130d876ea 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -32,6 +32,9 @@ struct ConnectionParameters; using ConnectionPtr = std::shared_ptr; using Connections = std::vector; +class NativeReader; +class NativeWriter; + /** Connection with database server, to use by client. * How to use - see Core/Protocol.h @@ -53,25 +56,9 @@ public: const String & client_name_, Protocol::Compression compression_, Protocol::Secure secure_, - Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0)) - : - host(host_), port(port_), default_database(default_database_), - user(user_), password(password_), - cluster(cluster_), - cluster_secret(cluster_secret_), - client_name(client_name_), - compression(compression_), - secure(secure_), - sync_request_timeout(sync_request_timeout_), - log_wrapper(*this) - { - /// Don't connect immediately, only on first need. + Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0)); - if (user.empty()) - user = "default"; - - setDescription(); - } + ~Connection() override; static ServerConnectionPtr createConnection(const ConnectionParameters & parameters, ContextPtr context); @@ -217,12 +204,12 @@ private: /// From where to read query execution result. std::shared_ptr maybe_compressed_in; - BlockInputStreamPtr block_in; - BlockInputStreamPtr block_logs_in; + std::unique_ptr block_in; + std::unique_ptr block_logs_in; /// Where to write data for INSERT. std::shared_ptr maybe_compressed_out; - BlockOutputStreamPtr block_out; + std::unique_ptr block_out; /// Logger is created lazily, for avoid to run DNS request in constructor. class LoggerWrapper @@ -261,7 +248,7 @@ private: Block receiveData(); Block receiveLogData(); - Block receiveDataImpl(BlockInputStreamPtr & stream); + Block receiveDataImpl(NativeReader & reader); std::vector receiveMultistringMessage(UInt64 msg_type) const; std::unique_ptr receiveException() const; diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeReader.cpp similarity index 91% rename from src/DataStreams/NativeBlockInputStream.cpp rename to src/DataStreams/NativeReader.cpp index 2fb661260f0..92c37a3f764 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeReader.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include @@ -23,17 +23,17 @@ namespace ErrorCodes } -NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_) +NativeReader::NativeReader(ReadBuffer & istr_, UInt64 server_revision_) : istr(istr_), server_revision(server_revision_) { } -NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_) +NativeReader::NativeReader(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_) : istr(istr_), header(header_), server_revision(server_revision_) { } -NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_, +NativeReader::NativeReader(ReadBuffer & istr_, UInt64 server_revision_, IndexForNativeFormat::Blocks::const_iterator index_block_it_, IndexForNativeFormat::Blocks::const_iterator index_block_end_) : istr(istr_), server_revision(server_revision_), @@ -57,7 +57,7 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server } // also resets few vars from IBlockInputStream (I didn't want to propagate resetParser upthere) -void NativeBlockInputStream::resetParser() +void NativeReader::resetParser() { istr_concrete = nullptr; use_index = false; @@ -67,11 +67,11 @@ void NativeBlockInputStream::resetParser() read_suffix_is_called = false; #endif - is_cancelled.store(false); - is_killed.store(false); + // is_cancelled.store(false); + // is_killed.store(false); } -void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) +void NativeReader::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { ISerialization::DeserializeBinaryBulkSettings settings; settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; @@ -91,13 +91,13 @@ void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column } -Block NativeBlockInputStream::getHeader() const +Block NativeReader::getHeader() const { return header; } -Block NativeBlockInputStream::readImpl() +Block NativeReader::read() { Block res; @@ -215,7 +215,7 @@ Block NativeBlockInputStream::readImpl() return res; } -void NativeBlockInputStream::updateAvgValueSizeHints(const Block & block) +void NativeReader::updateAvgValueSizeHints(const Block & block) { auto rows = block.rows(); if (rows < 10) diff --git a/src/DataStreams/NativeBlockInputStream.h b/src/DataStreams/NativeReader.h similarity index 86% rename from src/DataStreams/NativeBlockInputStream.h rename to src/DataStreams/NativeReader.h index 8f3d2843e0f..cfd58bde2cc 100644 --- a/src/DataStreams/NativeBlockInputStream.h +++ b/src/DataStreams/NativeReader.h @@ -57,32 +57,28 @@ struct IndexForNativeFormat * Can also be used to store data on disk. * In this case, can use the index. */ -class NativeBlockInputStream : public IBlockInputStream +class NativeReader { public: /// If a non-zero server_revision is specified, additional block information may be expected and read. - NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_); + NativeReader(ReadBuffer & istr_, UInt64 server_revision_); /// For cases when data structure (header) is known in advance. /// NOTE We may use header for data validation and/or type conversions. It is not implemented. - NativeBlockInputStream(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_); + NativeReader(ReadBuffer & istr_, const Block & header_, UInt64 server_revision_); /// For cases when we have an index. It allows to skip columns. Only columns specified in the index will be read. - NativeBlockInputStream(ReadBuffer & istr_, UInt64 server_revision_, + NativeReader(ReadBuffer & istr_, UInt64 server_revision_, IndexForNativeFormat::Blocks::const_iterator index_block_it_, IndexForNativeFormat::Blocks::const_iterator index_block_end_); - String getName() const override { return "Native"; } - static void readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); - Block getHeader() const override; + Block getHeader() const; void resetParser(); - -protected: - Block readImpl() override; + Block read(); private: ReadBuffer & istr; diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeWriter.cpp similarity index 95% rename from src/DataStreams/NativeBlockOutputStream.cpp rename to src/DataStreams/NativeWriter.cpp index 2bd41094147..6e26c443e29 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeWriter.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include @@ -20,7 +20,7 @@ namespace ErrorCodes } -NativeBlockOutputStream::NativeBlockOutputStream( +NativeWriter::NativeWriter( WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_, WriteBuffer * index_ostr_, size_t initial_size_of_file_) : ostr(ostr_), client_revision(client_revision_), header(header_), @@ -35,7 +35,7 @@ NativeBlockOutputStream::NativeBlockOutputStream( } -void NativeBlockOutputStream::flush() +void NativeWriter::flush() { ostr.next(); } @@ -62,7 +62,7 @@ static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuf } -void NativeBlockOutputStream::write(const Block & block) +void NativeWriter::write(const Block & block) { /// Additional information about the block. if (client_revision > 0) diff --git a/src/DataStreams/NativeBlockOutputStream.h b/src/DataStreams/NativeWriter.h similarity index 78% rename from src/DataStreams/NativeBlockOutputStream.h rename to src/DataStreams/NativeWriter.h index 7ff6c8dfa33..67fc179b620 100644 --- a/src/DataStreams/NativeBlockOutputStream.h +++ b/src/DataStreams/NativeWriter.h @@ -1,8 +1,8 @@ #pragma once -#include #include #include +#include namespace DB { @@ -17,20 +17,20 @@ class CompressedWriteBuffer; * A stream can be specified to write the index. The index contains offsets to each part of each column. * If an `append` is made to an existing file, and you need to write the index, then specify `initial_size_of_file`. */ -class NativeBlockOutputStream : public IBlockOutputStream +class NativeWriter { public: /** If non-zero client_revision is specified, additional block information can be written. */ - NativeBlockOutputStream( + NativeWriter( WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_ = false, WriteBuffer * index_ostr_ = nullptr, size_t initial_size_of_file_ = 0); - Block getHeader() const override { return header; } - void write(const Block & block) override; - void flush() override; + Block getHeader() const { return header; } + void write(const Block & block); + void flush(); - String getContentType() const override { return "application/octet-stream"; } + static String getContentType() { return "application/octet-stream"; } private: WriteBuffer & ostr; diff --git a/src/DataStreams/TemporaryFileStream.cpp b/src/DataStreams/TemporaryFileStream.cpp index a57ad6ed243..826cf5508d8 100644 --- a/src/DataStreams/TemporaryFileStream.cpp +++ b/src/DataStreams/TemporaryFileStream.cpp @@ -1,7 +1,7 @@ #include #include -#include -#include +#include +#include #include #include #include @@ -17,13 +17,13 @@ namespace DB TemporaryFileStream::TemporaryFileStream(const std::string & path) : file_in(path) , compressed_in(file_in) - , block_in(std::make_shared(compressed_in, DBMS_TCP_PROTOCOL_VERSION)) + , block_in(std::make_unique(compressed_in, DBMS_TCP_PROTOCOL_VERSION)) {} TemporaryFileStream::TemporaryFileStream(const std::string & path, const Block & header_) : file_in(path) , compressed_in(file_in) - , block_in(std::make_shared(compressed_in, header_, 0)) + , block_in(std::make_unique(compressed_in, header_, 0)) {} /// Flush data from input stream into file for future reading @@ -31,18 +31,15 @@ void TemporaryFileStream::write(const std::string & path, const Block & header, { WriteBufferFromFile file_buf(path); CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {})); - NativeBlockOutputStream output(compressed_buf, 0, header); + NativeWriter output(compressed_buf, 0, header); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); - output.writePrefix(); - Block block; while (executor.pull(block)) output.write(block); - output.writeSuffix(); compressed_buf.finalize(); } diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index f87e32eee48..c0c13605928 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -14,7 +15,7 @@ struct TemporaryFileStream { ReadBufferFromFile file_in; CompressedReadBuffer compressed_in; - BlockInputStreamPtr block_in; + std::unique_ptr block_in; explicit TemporaryFileStream(const std::string & path); TemporaryFileStream(const std::string & path, const Block & header_); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d9ae718dbd9..9901081d7dd 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -138,9 +137,6 @@ InputFormatPtr FormatFactory::getInput( UInt64 max_block_size, const std::optional & _format_settings) const { - if (name == "Native") - return std::make_shared(sample, buf); - auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); diff --git a/src/Formats/NativeFormat.cpp b/src/Formats/NativeFormat.cpp index f9cafbe5459..45a6853b3af 100644 --- a/src/Formats/NativeFormat.cpp +++ b/src/Formats/NativeFormat.cpp @@ -1,33 +1,96 @@ -#include -#include +#include +#include #include +#include +#include namespace DB { + +class NativeInputFormat final : public IInputFormat +{ +public: + NativeInputFormat(ReadBuffer & buf, const Block & header) + : IInputFormat(header, buf) + , reader(buf, header) {} + + String getName() const override { return "Native"; } + + void resetParser() override + { + reader.resetParser(); + } + + Chunk generate() override + { + auto block = reader.read(); + + assertBlocksHaveEqualStructure(getPort().getHeader(), block, getName()); + block.checkNumberOfRows(); + + size_t num_rows = block.rows(); + return Chunk(block.getColumns(), num_rows); + } + +private: + NativeReader reader; +}; + +class NativeOutputFormat final : public IOutputFormat +{ +public: + NativeOutputFormat(WriteBuffer & buf, const Block & header) + : IOutputFormat(header, buf) + , writer(buf, 0, header) + { + } + + String getName() const override { return "Native"; } + + std::string getContentType() const override + { + return writer.getContentType(); + } + +protected: + void consume(Chunk chunk) override + { + if (chunk) + { + + auto block = getPort(PortKind::Main).getHeader(); + block.setColumns(chunk.detachColumns()); + writer.write(block); + } + } + +private: + NativeWriter writer; +}; + void registerInputFormatNative(FormatFactory & factory) { - factory.registerInputFormat("Native", []( + factory.registerInputFormatProcessor("Native", []( ReadBuffer & buf, const Block & sample, - UInt64 /* max_block_size */, - FormatFactory::ReadCallback /* callback */, + const RowInputFormatParams &, const FormatSettings &) { - return std::make_shared(buf, sample, 0); + return std::make_shared(buf, sample); }); } void registerOutputFormatNative(FormatFactory & factory) { - factory.registerOutputFormat("Native", []( + factory.registerOutputFormatProcessor("Native", []( WriteBuffer & buf, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings &) { - return std::make_shared(buf, 0, sample); + return std::make_shared(buf, sample); }); } diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index de479c07bae..2686022980f 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -21,8 +21,6 @@ void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); void registerInputFormatNative(FormatFactory & factory); void registerOutputFormatNative(FormatFactory & factory); -void registerInputFormatProcessorNative(FormatFactory & factory); -void registerOutputFormatProcessorNative(FormatFactory & factory); void registerInputFormatProcessorRowBinary(FormatFactory & factory); void registerOutputFormatProcessorRowBinary(FormatFactory & factory); void registerInputFormatProcessorTabSeparated(FormatFactory & factory); @@ -96,8 +94,6 @@ void registerFormats() registerInputFormatNative(factory); registerOutputFormatNative(factory); - registerInputFormatProcessorNative(factory); - registerOutputFormatProcessorNative(factory); registerInputFormatProcessorRowBinary(factory); registerOutputFormatProcessorRowBinary(factory); registerInputFormatProcessorTabSeparated(factory); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 8763b04dde1..63e3577af55 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -1068,7 +1068,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co const std::string & path = file->path(); WriteBufferFromFile file_buf(path); CompressedWriteBuffer compressed_buf(file_buf); - NativeBlockOutputStream block_out(compressed_buf, DBMS_TCP_PROTOCOL_VERSION, getHeader(false)); + NativeWriter block_out(compressed_buf, DBMS_TCP_PROTOCOL_VERSION, getHeader(false)); LOG_DEBUG(log, "Writing part of aggregation data into temporary file {}.", path); ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart); @@ -1193,7 +1193,7 @@ template void Aggregator::writeToTemporaryFileImpl( AggregatedDataVariants & data_variants, Method & method, - IBlockOutputStream & out) const + NativeWriter & out) const { size_t max_temporary_block_size_rows = 0; size_t max_temporary_block_size_bytes = 0; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 1524453ab34..85ce83868c6 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -853,6 +853,7 @@ using ManyAggregatedDataVariants = std::vector; using ManyAggregatedDataVariantsPtr = std::shared_ptr; class CompiledAggregateFunctionsHolder; +class NativeWriter; /** How are "total" values calculated with WITH TOTALS? * (For more details, see TotalsHavingTransform.) @@ -1150,7 +1151,7 @@ private: void writeToTemporaryFileImpl( AggregatedDataVariants & data_variants, Method & method, - IBlockOutputStream & out) const; + NativeWriter & out) const; /// Merge NULL key data from hash table `src` into `dst`. template diff --git a/src/Processors/Formats/Impl/NativeFormat.h b/src/Processors/Formats/Impl/NativeFormat.h deleted file mode 100644 index 4757274e1c9..00000000000 --- a/src/Processors/Formats/Impl/NativeFormat.h +++ /dev/null @@ -1,179 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include - - -namespace DB -{ - -class NativeInputFormatFromNativeBlockInputStream : public IInputFormat -{ -public: - NativeInputFormatFromNativeBlockInputStream(const Block & header, ReadBuffer & in_) - : IInputFormat(header, in_) - , stream(std::make_shared(*in, header, 0)) - { - } - - String getName() const override { return "NativeInputFormatFromNativeBlockInputStream"; } - -protected: - void resetParser() override - { - IInputFormat::resetParser(); - stream->resetParser(); - read_prefix = false; - read_suffix = false; - } - - - Chunk generate() override - { - /// TODO: do something with totals and extremes. - - if (!read_prefix) - { - stream->readPrefix(); - read_prefix = true; - } - - auto block = stream->read(); - if (!block) - { - if (!read_suffix) - { - stream->readSuffix(); - read_suffix = true; - } - - return Chunk(); - } - - assertBlocksHaveEqualStructure(getPort().getHeader(), block, getName()); - block.checkNumberOfRows(); - - UInt64 num_rows = block.rows(); - return Chunk(block.getColumns(), num_rows); - } - -private: - std::shared_ptr stream; - bool read_prefix = false; - bool read_suffix = false; -}; - - -class NativeOutputFormatFromNativeBlockOutputStream : public IOutputFormat -{ -public: - NativeOutputFormatFromNativeBlockOutputStream(const Block & header, WriteBuffer & out_) - : IOutputFormat(header, out_) - , stream(std::make_shared(out, 0, header)) - { - } - - String getName() const override { return "NativeOutputFormatFromNativeBlockOutputStream"; } - - void setRowsBeforeLimit(size_t rows_before_limit) override - { - stream->setRowsBeforeLimit(rows_before_limit); - } - - void onProgress(const Progress & progress) override - { - stream->onProgress(progress); - } - - std::string getContentType() const override - { - return stream->getContentType(); - } - -protected: - void consume(Chunk chunk) override - { - writePrefixIfNot(); - - if (chunk) - { - - auto block = getPort(PortKind::Main).getHeader(); - block.setColumns(chunk.detachColumns()); - stream->write(block); - } - } - - void consumeTotals(Chunk chunk) override - { - writePrefixIfNot(); - - auto block = getPort(PortKind::Totals).getHeader(); - block.setColumns(chunk.detachColumns()); - stream->setTotals(block); - } - - void consumeExtremes(Chunk chunk) override - { - writePrefixIfNot(); - - auto block = getPort(PortKind::Extremes).getHeader(); - block.setColumns(chunk.detachColumns()); - stream->setExtremes(block); - } - - void finalize() override - { - writePrefixIfNot(); - writeSuffixIfNot(); - } - -private: - std::shared_ptr stream; - bool prefix_written = false; - bool suffix_written = false; - - void writePrefixIfNot() - { - if (!prefix_written) - stream->writePrefix(); - - prefix_written = true; - } - - void writeSuffixIfNot() - { - if (!suffix_written) - stream->writeSuffix(); - - suffix_written = true; - } -}; - -void registerInputFormatProcessorNative(FormatFactory & factory) -{ - factory.registerInputFormatProcessor("Native", []( - ReadBuffer & buf, - const Block & sample, - const RowInputFormatParams &, - const FormatSettings &) - { - return std::make_shared(sample, buf); - }); -} - -void registerOutputFormatProcessorNative(FormatFactory & factory) -{ - factory.registerOutputFormatProcessor("Native", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams &, - const FormatSettings &) - { - return std::make_shared(sample, buf); - }); -} - -} diff --git a/src/Processors/Sources/SourceFromSingleChunk.cpp b/src/Processors/Sources/SourceFromSingleChunk.cpp new file mode 100644 index 00000000000..776ed98599f --- /dev/null +++ b/src/Processors/Sources/SourceFromSingleChunk.cpp @@ -0,0 +1,26 @@ +#include +#include +#include + +namespace DB +{ + +SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : SourceWithProgress(std::move(header)), chunk(std::move(chunk_)) {} +SourceFromSingleChunk::SourceFromSingleChunk(Block data) : SourceWithProgress(data.cloneEmpty()), chunk(data.getColumns(), data.rows()) +{ + const auto & sample = getPort().getHeader(); + bool has_aggregate_functions = false; + for (auto & type : sample.getDataTypes()) + if (typeid_cast(type.get())) + has_aggregate_functions = true; + + if (has_aggregate_functions) + { + auto info = std::make_shared(); + info->bucket_num = data.info.bucket_num; + info->is_overflows = data.info.is_overflows; + chunk.setChunkInfo(std::move(info)); + } +} + +} diff --git a/src/Processors/Sources/SourceFromSingleChunk.h b/src/Processors/Sources/SourceFromSingleChunk.h index 8268fa5b0a6..e06387b556f 100644 --- a/src/Processors/Sources/SourceFromSingleChunk.h +++ b/src/Processors/Sources/SourceFromSingleChunk.h @@ -8,8 +8,8 @@ namespace DB class SourceFromSingleChunk : public SourceWithProgress { public: - explicit SourceFromSingleChunk(Block header, Chunk chunk_) : SourceWithProgress(std::move(header)), chunk(std::move(chunk_)) {} - explicit SourceFromSingleChunk(Block data) : SourceWithProgress(data.cloneEmpty()), chunk(data.getColumns(), data.rows()) {} + explicit SourceFromSingleChunk(Block header, Chunk chunk_); + explicit SourceFromSingleChunk(Block data); String getName() const override { return "SourceFromSingleChunk"; } protected: diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 7e5480c37ca..9011d188b81 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include @@ -56,9 +56,8 @@ namespace public: SourceFromNativeStream(const Block & header, const std::string & path) : ISource(header), file_in(path), compressed_in(file_in), - block_in(std::make_shared(compressed_in, DBMS_TCP_PROTOCOL_VERSION)) + block_in(std::make_unique(compressed_in, DBMS_TCP_PROTOCOL_VERSION)) { - block_in->readPrefix(); } String getName() const override { return "SourceFromNativeStream"; } @@ -71,7 +70,6 @@ namespace auto block = block_in->read(); if (!block) { - block_in->readSuffix(); block_in.reset(); return {}; } @@ -82,7 +80,7 @@ namespace private: ReadBufferFromFile file_in; CompressedReadBuffer compressed_in; - BlockInputStreamPtr block_in; + std::unique_ptr block_in; }; } diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index ca78a29071e..6e379a3c4ba 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -6,8 +6,8 @@ #include #include #include -#include -#include +#include +#include #include @@ -33,11 +33,10 @@ public: BufferingToFileTransform(const Block & header, Poco::Logger * log_, std::string path_) : IAccumulatingTransform(header, header), log(log_) , path(std::move(path_)), file_buf_out(path), compressed_buf_out(file_buf_out) - , out_stream(std::make_shared(compressed_buf_out, 0, header)) + , out_stream(std::make_unique(compressed_buf_out, 0, header)) { LOG_INFO(log, "Sorting and writing part of data into temporary file {}", path); ProfileEvents::increment(ProfileEvents::ExternalSortWritePart); - out_stream->writePrefix(); } String getName() const override { return "BufferingToFileTransform"; } @@ -51,7 +50,6 @@ public: { if (out_stream) { - out_stream->writeSuffix(); compressed_buf_out.next(); file_buf_out.next(); LOG_INFO(log, "Done writing part of data into temporary file {}", path); @@ -60,7 +58,7 @@ public: file_in = std::make_unique(path); compressed_in = std::make_unique(*file_in); - block_in = std::make_shared(*compressed_in, getOutputPort().getHeader(), 0); + block_in = std::make_unique(*compressed_in, getOutputPort().getHeader(), 0); } if (!block_in) @@ -69,7 +67,6 @@ public: auto block = block_in->read(); if (!block) { - block_in->readSuffix(); block_in.reset(); return {}; } @@ -83,11 +80,11 @@ private: std::string path; WriteBufferFromFile file_buf_out; CompressedWriteBuffer compressed_buf_out; - BlockOutputStreamPtr out_stream; + std::unique_ptr out_stream; std::unique_ptr file_in; std::unique_ptr compressed_in; - BlockInputStreamPtr block_in; + std::unique_ptr block_in; }; MergeSortingTransform::MergeSortingTransform( diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 11f23530c9e..2c9098adaa6 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -9,8 +9,8 @@ #include #include -#include -#include +#include +#include namespace ProfileEvents diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cbb055e956c..f3247e7bc2b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -16,8 +16,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -1369,7 +1369,7 @@ bool TCPHandler::receiveUnexpectedData(bool throw_exception) else maybe_compressed_in = in; - auto skip_block_in = std::make_shared(*maybe_compressed_in, client_tcp_protocol_version); + auto skip_block_in = std::make_shared(*maybe_compressed_in, client_tcp_protocol_version); bool read_ok = skip_block_in->read(); if (!read_ok) @@ -1399,7 +1399,7 @@ void TCPHandler::initBlockInput() else if (state.need_receive_data_for_input) header = state.input_header; - state.block_in = std::make_shared( + state.block_in = std::make_unique( *state.maybe_compressed_in, header, client_tcp_protocol_version); @@ -1430,7 +1430,7 @@ void TCPHandler::initBlockOutput(const Block & block) state.maybe_compressed_out = out; } - state.block_out = std::make_shared( + state.block_out = std::make_unique( *state.maybe_compressed_out, client_tcp_protocol_version, block.cloneEmpty(), @@ -1444,7 +1444,7 @@ void TCPHandler::initLogsBlockOutput(const Block & block) { /// Use uncompressed stream since log blocks usually contain only one row const Settings & query_settings = query_context->getSettingsRef(); - state.logs_block_out = std::make_shared( + state.logs_block_out = std::make_unique( *out, client_tcp_protocol_version, block.cloneEmpty(), diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 624031b60b7..d001b12ee66 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include "IServer.h" @@ -44,15 +46,15 @@ struct QueryState /// destroyed after input/output blocks, because they may contain other /// threads that use this queue. InternalTextLogsQueuePtr logs_queue; - BlockOutputStreamPtr logs_block_out; + std::unique_ptr logs_block_out; /// From where to read data for INSERT. std::shared_ptr maybe_compressed_in; - BlockInputStreamPtr block_in; + std::unique_ptr block_in; /// Where to write result data. std::shared_ptr maybe_compressed_out; - BlockOutputStreamPtr block_out; + std::unique_ptr block_out; Block block_for_insert; /// Query text. diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 433776ffb08..167e36ebbe3 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -189,7 +189,7 @@ namespace if (header_buf.hasPendingData()) { - NativeBlockInputStream header_block_in(header_buf, DBMS_TCP_PROTOCOL_VERSION); + NativeReader header_block_in(header_buf, DBMS_TCP_PROTOCOL_VERSION); distributed_header.block_header = header_block_in.read(); if (!distributed_header.block_header) throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read header from the {} batch", in.getFileName()); @@ -268,8 +268,7 @@ namespace void writeAndConvert(RemoteInserter & remote, ReadBufferFromFile & in) { CompressedReadBuffer decompressing_in(in); - NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); - block_in.readPrefix(); + NativeReader block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); while (Block block = block_in.read()) { @@ -282,8 +281,6 @@ namespace converting_actions->execute(block); remote.write(block); } - - block_in.readSuffix(); } void writeRemoteConvert( @@ -909,7 +906,7 @@ public: { std::unique_ptr in; std::unique_ptr decompressing_in; - std::unique_ptr block_in; + std::unique_ptr block_in; Poco::Logger * log = nullptr; @@ -919,12 +916,11 @@ public: { in = std::make_unique(file_name); decompressing_in = std::make_unique(*in); - block_in = std::make_unique(*decompressing_in, DBMS_TCP_PROTOCOL_VERSION); + block_in = std::make_unique(*decompressing_in, DBMS_TCP_PROTOCOL_VERSION); log = &Poco::Logger::get("DirectoryMonitorSource"); readDistributedHeader(*in, log); - block_in->readPrefix(); first_block = block_in->read(); } @@ -957,10 +953,7 @@ protected: auto block = data.block_in->read(); if (!block) - { - data.block_in->readSuffix(); return {}; - } size_t num_rows = block.rows(); return Chunk(block.getColumns(), num_rows); @@ -1048,8 +1041,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map LOG_DEBUG(log, "Processing batch {} with old format (no header/rows)", in.getFileName()); CompressedReadBuffer decompressing_in(in); - NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); - block_in.readPrefix(); + NativeReader block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); while (Block block = block_in.read()) { @@ -1059,7 +1051,6 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map if (!header) header = block.cloneEmpty(); } - block_in.readSuffix(); } } catch (const Exception & e) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index ad828e786b4..1841be22b72 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -707,7 +707,7 @@ void DistributedSink::writeToShard(const Block & block, const std::vector -#include +#include #include #include #include @@ -222,7 +222,7 @@ void Service::sendPartFromMemory( writeStringBinary(name, out); projection->checksums.write(out); - NativeBlockOutputStream block_out(out, 0, projection_sample_block); + NativeWriter block_out(out, 0, projection_sample_block); block_out.write(part_in_memory->block); } @@ -230,7 +230,7 @@ void Service::sendPartFromMemory( if (!part_in_memory) throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); - NativeBlockOutputStream block_out(out, 0, metadata_snapshot->getSampleBlock()); + NativeWriter block_out(out, 0, metadata_snapshot->getSampleBlock()); part->checksums.write(out); block_out.write(part_in_memory->block); @@ -569,7 +569,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( if (!checksums.read(in)) throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA); - NativeBlockInputStream block_in(in, 0); + NativeReader block_in(in, 0); auto block = block_in.read(); throttler->add(block.bytes()); @@ -599,7 +599,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( if (!checksums.read(in)) throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA); - NativeBlockInputStream block_in(in, 0); + NativeReader block_in(in, 0); auto block = block_in.read(); throttler->add(block.bytes()); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 2c1d785236c..28f25296a7b 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -57,7 +57,7 @@ void MergeTreeWriteAheadLog::init() /// Small hack: in NativeBlockOutputStream header is used only in `getHeader` method. /// To avoid complex logic of changing it during ALTERs we leave it empty. - block_out = std::make_unique(*out, 0, Block{}); + block_out = std::make_unique(*out, 0, Block{}); min_block_number = std::numeric_limits::max(); max_block_number = -1; bytes_at_last_sync = 0; @@ -119,7 +119,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor MergeTreeData::MutableDataPartsVector parts; auto in = disk->readFile(path, {}, 0); - NativeBlockInputStream block_in(*in, 0); + NativeReader block_in(*in, 0); NameSet dropped_parts; while (!in->eof()) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 8d1ea3c332e..7624dc303e0 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include +#include #include #include #include @@ -78,7 +78,7 @@ private: String path; std::unique_ptr out; - std::unique_ptr block_out; + std::unique_ptr block_out; Int64 min_block_number = std::numeric_limits::max(); Int64 max_block_number = -1; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 2acdba18c2d..1353c8d67aa 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -107,7 +107,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context) auto backup_buf = disk->writeFile(path + tmp_backup_file_name); auto compressed_backup_buf = CompressedWriteBuffer(*backup_buf); - auto backup_stream = NativeBlockOutputStream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); + auto backup_stream = NativeWriter(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()); auto new_data = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index fe55123335a..58c56f1401f 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -54,7 +54,7 @@ private: String backup_file_name; std::unique_ptr backup_buf; CompressedWriteBuffer compressed_backup_buf; - NativeBlockOutputStream backup_stream; + NativeWriter backup_stream; bool persistent; }; @@ -212,19 +212,20 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) { auto backup_buf = disk->readFile(file_path); CompressedReadBuffer compressed_backup_buf(*backup_buf); - NativeBlockInputStream backup_stream(compressed_backup_buf, 0); - - backup_stream.readPrefix(); + NativeReader backup_stream(compressed_backup_buf, 0); + BlockStreamProfileInfo info; while (Block block = backup_stream.read()) + { + info.update(block); insertBlock(block); + } finishInsert(); - backup_stream.readSuffix(); /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. LOG_INFO(&Poco::Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", - file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize()); + file_path, info.rows, ReadableSize(info.bytes), getSize()); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index f1ab365e458..2dc2577f245 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -15,8 +15,8 @@ #include #include -#include -#include +#include +#include #include @@ -136,7 +136,7 @@ private: */ bool started = false; std::optional data_in; - std::optional block_in; + std::optional block_in; void start() { @@ -214,7 +214,6 @@ public: if (done) return; - block_out.writeSuffix(); data_out->next(); data_out_compressed->next(); data_out_compressed->finalize(); @@ -245,7 +244,7 @@ private: String index_out_file; std::unique_ptr index_out_compressed; std::unique_ptr index_out; - NativeBlockOutputStream block_out; + NativeWriter block_out; bool done = false; }; diff --git a/utils/wal-dump/main.cpp b/utils/wal-dump/main.cpp index 361aa9df887..0e47c39fb5a 100644 --- a/utils/wal-dump/main.cpp +++ b/utils/wal-dump/main.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -21,7 +21,7 @@ static void dump(const std::string & bin_path) { DB::ReadBufferFromFile in(bin_path); - DB::NativeBlockInputStream block_in(in, 0); + DB::NativeReader block_in(in, 0); DB::Block block; DB::WriteBufferFromFileDescriptor out(STDOUT_FILENO); From f953cb85f850da07bb5316af5007d51bf4bee1fe Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Fri, 8 Oct 2021 21:01:14 +0300 Subject: [PATCH 308/950] Check --- .gitmodules | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..d8c5aa640a8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -212,7 +212,8 @@ url = https://github.com/ClickHouse-Extras/libpq [submodule "contrib/boringssl"] path = contrib/boringssl - url = https://github.com/ClickHouse-Extras/boringssl.git + url = https://github.com/FArthur-cmd/boringssl.git + branch = update_BoringSSL [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git From 1f6d5482b1452673275d19d1423b01956665e95e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 21:33:51 +0300 Subject: [PATCH 309/950] Fix some tests. --- src/Formats/NativeFormat.cpp | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Formats/NativeFormat.cpp b/src/Formats/NativeFormat.cpp index 45a6853b3af..b94cb279412 100644 --- a/src/Formats/NativeFormat.cpp +++ b/src/Formats/NativeFormat.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -14,18 +15,21 @@ class NativeInputFormat final : public IInputFormat public: NativeInputFormat(ReadBuffer & buf, const Block & header) : IInputFormat(header, buf) - , reader(buf, header) {} + , reader(buf, header, 0) {} String getName() const override { return "Native"; } void resetParser() override { + IInputFormat::resetParser(); reader.resetParser(); } Chunk generate() override { auto block = reader.read(); + if (!block) + return {}; assertBlocksHaveEqualStructure(getPort().getHeader(), block, getName()); block.checkNumberOfRows(); @@ -59,8 +63,16 @@ protected: { if (chunk) { - auto block = getPort(PortKind::Main).getHeader(); + + // const auto & info = chunk.getChunkInfo(); + // const auto * agg_info = typeid_cast(info.get()); + // if (agg_info) + // { + // block.info.bucket_num = agg_info->bucket_num; + // block.info.is_overflows = agg_info->is_overflows; + // } + block.setColumns(chunk.detachColumns()); writer.write(block); } From 21bdc12b9c7269036b6ca62425a90b98c0422111 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 21:38:12 +0300 Subject: [PATCH 310/950] Pull libcxx --- contrib/libcxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libcxx b/contrib/libcxx index 2fa892f69ac..61e60294b1d 160000 --- a/contrib/libcxx +++ b/contrib/libcxx @@ -1 +1 @@ -Subproject commit 2fa892f69acbaa40f8a18c6484854a6183a34482 +Subproject commit 61e60294b1de01483caa9f5d00f437c99b674de6 From 2ad2f20176bb16d536217c0ad1f1737fbd1479fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 21:55:09 +0300 Subject: [PATCH 311/950] Add NOLINT --- src/Core/examples/coro.cpp | 58 +++++++++++++++++++------------------- 1 file changed, 29 insertions(+), 29 deletions(-) diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp index a9728314495..e3a990e228d 100644 --- a/src/Core/examples/coro.cpp +++ b/src/Core/examples/coro.cpp @@ -27,11 +27,11 @@ namespace std template -struct suspend_value +struct suspend_value // NOLINT(readability-identifier-naming) { - constexpr bool await_ready() const noexcept { return true; } - constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} - constexpr T await_resume() const noexcept + constexpr bool await_ready() const noexcept { return true; } // NOLINT(readability-identifier-naming) + constexpr void await_suspend(std::coroutine_handle<>) const noexcept {} // NOLINT(readability-identifier-naming) + constexpr T await_resume() const noexcept // NOLINT(readability-identifier-naming) { std::cout << " ret " << val << std::endl; return val; @@ -41,57 +41,57 @@ struct suspend_value }; template -struct resumable +struct Task { - struct promise_type + struct promise_type // NOLINT(readability-identifier-naming) { using coro_handle = std::coroutine_handle; - auto get_return_object() { return coro_handle::from_promise(*this); } - auto initial_suspend() { return std::suspend_never(); } - auto final_suspend() noexcept { return suspend_value{*r->value}; } + auto get_return_object() { return coro_handle::from_promise(*this); } // NOLINT(readability-identifier-naming) + auto initial_suspend() { return std::suspend_never(); } // NOLINT(readability-identifier-naming) + auto final_suspend() noexcept { return suspend_value{*r->value}; } // NOLINT(readability-identifier-naming) //void return_void() {} - void return_value(T value_) { r->value = value_; } - void unhandled_exception() + void return_value(T value_) { r->value = value_; } // NOLINT(readability-identifier-naming) + void unhandled_exception() // NOLINT(readability-identifier-naming) { DB::tryLogCurrentException("Logger"); - r->exception = std::current_exception(); + r->exception = std::current_exception(); // NOLINT(bugprone-throw-keyword-missing) } explicit promise_type(std::string tag_) : tag(tag_) {} ~promise_type() { std::cout << "~promise_type " << tag << std::endl; } std::string tag; coro_handle next; - resumable * r = nullptr; + Task * r = nullptr; }; using coro_handle = std::coroutine_handle; - bool await_ready() const noexcept { return false; } - void await_suspend(coro_handle g) noexcept + bool await_ready() const noexcept { return false; } // NOLINT(readability-identifier-naming) + void await_suspend(coro_handle g) noexcept // NOLINT(readability-identifier-naming) { std::cout << " await_suspend " << my.promise().tag << std::endl; std::cout << " g tag " << g.promise().tag << std::endl; g.promise().next = my; } - T await_resume() noexcept + T await_resume() noexcept // NOLINT(readability-identifier-naming) { std::cout << " await_res " << my.promise().tag << std::endl; return *value; } - resumable(coro_handle handle) : my(handle), tag(handle.promise().tag) + explicit Task(coro_handle handle) : my(handle), tag(handle.promise().tag) { assert(handle); my.promise().r = this; - std::cout << " resumable " << tag << std::endl; + std::cout << " Task " << tag << std::endl; } - resumable(resumable &) = delete; - resumable(resumable &&rhs) : my(rhs.my), tag(rhs.tag) + Task(Task &) = delete; + Task(Task &&rhs) : my(rhs.my), tag(rhs.tag) { rhs.my = {}; - std::cout << " resumable&& " << tag << std::endl; + std::cout << " Task&& " << tag << std::endl; } - static bool resume_impl(resumable *r) + static bool resumeImpl(Task *r) { if (r->value) return false; @@ -100,7 +100,7 @@ struct resumable if (next) { - if (resume_impl(next.promise().r)) + if (resumeImpl(next.promise().r)) return true; next = {}; } @@ -116,7 +116,7 @@ struct resumable bool resume() { - return resume_impl(this); + return resumeImpl(this); } T res() @@ -124,9 +124,9 @@ struct resumable return *value; } - ~resumable() + ~Task() { - std::cout << " ~resumable " << tag << std::endl; + std::cout << " ~Task " << tag << std::endl; } private: @@ -136,7 +136,7 @@ private: std::exception_ptr exception; }; -resumable boo([[maybe_unused]] std::string tag) +Task boo([[maybe_unused]] std::string tag) { std::cout << "x" << std::endl; co_await std::suspend_always(); @@ -145,7 +145,7 @@ resumable boo([[maybe_unused]] std::string tag) co_return 1; } -resumable bar([[maybe_unused]] std::string tag) +Task bar([[maybe_unused]] std::string tag) { std::cout << "a" << std::endl; int res1 = co_await boo("boo1"); @@ -157,7 +157,7 @@ resumable bar([[maybe_unused]] std::string tag) co_return res1 + res2; // 1 + 1 = 2 } -resumable foo([[maybe_unused]] std::string tag) { +Task foo([[maybe_unused]] std::string tag) { std::cout << "Hello" << std::endl; auto res1 = co_await bar("bar1"); std::cout << "Coro " << res1 << std::endl; From 75af011068122162c87722a05ddb80d30508c6f2 Mon Sep 17 00:00:00 2001 From: vesslanjin Date: Fri, 8 Oct 2021 14:58:23 -0400 Subject: [PATCH 312/950] use while instead of popcnt + for Co-authored-by: Zhu Jasper jasper.zhu@intel.com --- src/Columns/ColumnFixedString.cpp | 9 ++------- src/Columns/ColumnVector.cpp | 9 ++------- src/Columns/ColumnsCommon.cpp | 9 ++------- 3 files changed, 6 insertions(+), 21 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 1080f29b5aa..ea66ba73d8e 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -248,19 +248,14 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); mask = ~mask; - if (0 == mask) - { - /// Nothing is inserted. - } - else if (0xFFFF == mask) + if (0xFFFF == mask) { res->chars.insert(data_pos, data_pos + chars_per_simd_elements); } else { size_t res_chars_size = res->chars.size(); - size_t pcnt = __builtin_popcount(mask); - for (size_t j = 0; j < pcnt; ++j) + while(mask) { size_t index = __builtin_ctz(mask); res->chars.resize(res_chars_size + n); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 61ba2074bd7..f45b0581213 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -327,18 +327,13 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); mask = ~mask; - if (0 == mask) - { - /// Nothing is inserted. - } - else if (0xFFFF == mask) + if (0xFFFF == mask) { res_data.insert(data_pos, data_pos + SIMD_BYTES); } else { - size_t pcnt = __builtin_popcount(mask); - for (size_t j = 0; j < pcnt; ++j) + while(mask) { size_t index = __builtin_ctz(mask); res_data.push_back(data_pos[index]); diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index b4614abe490..8a6f5c24a32 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -241,11 +241,7 @@ namespace zero_vec)); mask = ~mask; - if (mask == 0) - { - /// SIMD_BYTES consecutive rows do not pass the filter - } - else if (mask == 0xffff) + if (mask == 0xffff) { /// SIMD_BYTES consecutive rows pass the filter const auto first = offsets_pos == offsets_begin; @@ -262,8 +258,7 @@ namespace } else { - size_t pcnt = __builtin_popcount(mask); - for (size_t j = 0; j < pcnt; ++j) + while(mask) { size_t index = __builtin_ctz(mask); copy_array(offsets_pos + index); From 7a5cc357ecb6df1ec14b008dbc04d8a14de1776a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 8 Oct 2021 22:44:08 +0300 Subject: [PATCH 313/950] Fix style --- src/Core/examples/coro.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp index e3a990e228d..e11d0af9e22 100644 --- a/src/Core/examples/coro.cpp +++ b/src/Core/examples/coro.cpp @@ -1,5 +1,3 @@ -#include - #include #include #include @@ -157,7 +155,8 @@ Task bar([[maybe_unused]] std::string tag) co_return res1 + res2; // 1 + 1 = 2 } -Task foo([[maybe_unused]] std::string tag) { +Task foo([[maybe_unused]] std::string tag) +{ std::cout << "Hello" << std::endl; auto res1 = co_await bar("bar1"); std::cout << "Coro " << res1 << std::endl; From 8513994a659fd287a3a890fcd413cc23cc3d061d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 8 Oct 2021 23:51:21 +0300 Subject: [PATCH 314/950] add logging in ZooKeeper client --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 46 ++++++++++++++++---------- src/Common/ZooKeeper/ZooKeeperImpl.h | 6 ++-- 2 files changed, 32 insertions(+), 20 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index d79a94169b2..cf607a3d70e 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -289,7 +289,7 @@ ZooKeeper::~ZooKeeper() { try { - finalize(false, false); + finalize(false, false, "destructor called"); if (send_thread.joinable()) send_thread.join(); @@ -299,7 +299,7 @@ ZooKeeper::~ZooKeeper() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } @@ -317,6 +317,7 @@ ZooKeeper::ZooKeeper( session_timeout(session_timeout_), operation_timeout(std::min(operation_timeout_, session_timeout_)) { + log = &Poco::Logger::get("ZooKeeperClient"); std::atomic_store(&zk_log, std::move(zk_log_)); if (!root_path.empty()) @@ -450,6 +451,10 @@ void ZooKeeper::connect( message << fail_reasons.str() << "\n"; throw Exception(message.str(), Error::ZCONNECTIONLOSS); } + else + { + LOG_TEST(log, "Connected to ZooKeeper at {} with session_id {}", socket.peerAddress().toString(), session_id); + } } @@ -604,8 +609,8 @@ void ZooKeeper::sendThread() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); - finalize(true, false); + tryLogCurrentException(log); + finalize(true, false, "exception in sendThread"); } } @@ -663,8 +668,8 @@ void ZooKeeper::receiveThread() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); - finalize(false, true); + tryLogCurrentException(log); + finalize(false, true, "exception in receiveThread"); } } @@ -799,7 +804,7 @@ void ZooKeeper::receiveEvent() } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); /// Unrecoverable. Don't leave incorrect state in memory. if (!response) @@ -819,7 +824,7 @@ void ZooKeeper::receiveEvent() catch (...) { /// Throw initial exception, not exception from callback. - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } throw; @@ -832,10 +837,15 @@ void ZooKeeper::receiveEvent() } -void ZooKeeper::finalize(bool error_send, bool error_receive) +void ZooKeeper::finalize(bool error_send, bool error_receive, const String & reason) { /// If some thread (send/receive) already finalizing session don't try to do it - if (finalization_started.exchange(true)) + bool already_started = finalization_started.exchange(true); + + LOG_TEST(log, "Finalizing session {}: finalization_started={}, queue_closed={}, reason={}", + session_id, already_started, requests_queue.isClosed(), reason); + + if (already_started) return; auto expire_session_if_not_expired = [&] @@ -860,7 +870,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) /// This happens for example, when "Cannot push request to queue within operation timeout". /// Just mark session expired in case of error on close request, otherwise sendThread may not stop. expire_session_if_not_expired(); - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } /// Send thread will exit after sending close request or on expired flag @@ -879,7 +889,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) catch (...) { /// We must continue to execute all callbacks, because the user is waiting for them. - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } if (!error_receive && receive_thread.joinable()) @@ -908,7 +918,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) catch (...) { /// We must continue to all other callbacks, because the user is waiting for them. - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } } @@ -939,7 +949,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } } @@ -967,7 +977,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } } @@ -983,14 +993,14 @@ void ZooKeeper::finalize(bool error_send, bool error_receive) } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } } } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + tryLogCurrentException(log); } } @@ -1028,7 +1038,7 @@ void ZooKeeper::pushRequest(RequestInfo && info) } catch (...) { - finalize(false, false); + finalize(false, false, getCurrentExceptionMessage(false, false, false)); throw; } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index ce37ca7b650..53908e5b0c7 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -187,7 +187,7 @@ public: /// it will do read in another session, that read may not see the /// already performed write. - void finalize() override { finalize(false, false); } + void finalize() override { finalize(false, false, "unknown"); } void setZooKeeperLog(std::shared_ptr zk_log_); @@ -240,6 +240,8 @@ private: ThreadFromGlobalPool send_thread; ThreadFromGlobalPool receive_thread; + Poco::Logger * log; + void connect( const Nodes & node, Poco::Timespan connection_timeout); @@ -257,7 +259,7 @@ private: void close(); /// Call all remaining callbacks and watches, passing errors to them. - void finalize(bool error_send, bool error_receive); + void finalize(bool error_send, bool error_receive, const String & reason); template void write(const T &); From 5d6da023bbf590f511a8be1c59d280eba757b068 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Oct 2021 01:42:08 +0300 Subject: [PATCH 315/950] clickhouse-test: fix hung check under stress tests https://clickhouse-test-reports.s3.yandex.net/29856/e2d6698244d43979b3fe2478dfdcd8dc3a91a0fd/stress_test_(address).html#fail1 --- tests/clickhouse-test | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6bbfa97ab66..f00e5574c67 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -87,8 +87,11 @@ class Client(clickhouse_driver.Client): return data # Helpers -def make_clickhouse_client(base_args, *args, **kwargs): +def make_clickhouse_client(base_args): return Client(host=base_args.tcp_host, port=base_args.tcp_port, + # hung check in stress tests may remove the database, + # hence we should use 'system'. + database='system', settings=get_additional_client_options_dict(base_args)) def clickhouse_execute_one(base_args, *args, **kwargs): return make_clickhouse_client(base_args).execute_one(*args, **kwargs) From 42ca2b4bb241827edf69bbd6938d6b19c31935f1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Oct 2021 01:43:00 +0300 Subject: [PATCH 316/950] clickhouse-test: remove not existing options for pylint https://clickhouse-test-reports.s3.yandex.net/29856/e2d6698244d43979b3fe2478dfdcd8dc3a91a0fd/style_check/test_run.txt.out.log --- tests/clickhouse-test | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f00e5574c67..f10e38b87e5 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1,11 +1,7 @@ #!/usr/bin/env python3 # pylint: disable=too-many-return-statements -# pylint: disable=consider-using-f-string # pylint: disable=global-variable-not-assigned -# pylint: disable=consider-using-with -# pylint: disable=unspecified-encoding -# pylint: disable=consider-using-min-builtin import enum import shutil From 860f74db623ac3f00aa2811e0de9154fd0f777bc Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Sat, 9 Oct 2021 11:15:13 +0800 Subject: [PATCH 317/950] remove unused var --- src/Storages/StorageS3Cluster.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e4ae27e43bf..3314571c565 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -89,7 +89,6 @@ Pipe StorageS3Cluster::read( StorageS3::updateClientAndAuthSettings(context, client_auth); auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings()); - S3::URI s3_uri(Poco::URI{filename}); StorageS3::updateClientAndAuthSettings(context, client_auth); auto iterator = std::make_shared(*client_auth.client, client_auth.uri); From 89a52930c68e6c256aa9fe959dd4bb33aaddfc85 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 9 Oct 2021 11:47:08 +0800 Subject: [PATCH 318/950] Cosmetic refactoring of server constants. --- src/Functions/FunctionConstantBase.h | 17 ++++------------- src/Functions/serverConstants.cpp | 19 +++++++++++-------- 2 files changed, 15 insertions(+), 21 deletions(-) diff --git a/src/Functions/FunctionConstantBase.h b/src/Functions/FunctionConstantBase.h index 35096a9942f..2d237c77256 100644 --- a/src/Functions/FunctionConstantBase.h +++ b/src/Functions/FunctionConstantBase.h @@ -12,18 +12,9 @@ template class FunctionConstantBase : public IFunction { public: - - /// For server-level constants (uptime(), version(), etc) - explicit FunctionConstantBase(ContextPtr context, T && constant_value_) - : is_distributed(context->isDistributed()) - , constant_value(std::forward(constant_value_)) - { - } - - /// For real constants (pi(), e(), etc) - explicit FunctionConstantBase(const T & constant_value_) - : is_distributed(false) - , constant_value(constant_value_) + template + explicit FunctionConstantBase(U && constant_value_, bool is_distributed_ = false) + : constant_value(std::forward(constant_value_)), is_distributed(is_distributed_) { } @@ -56,8 +47,8 @@ public: } private: - bool is_distributed; const T constant_value; + bool is_distributed; }; } diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 9a53a5cf582..49d45368439 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -24,7 +24,7 @@ namespace public: static constexpr auto name = "buildId"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionBuildId(ContextPtr context) : FunctionConstantBase(context, SymbolIndex::instance()->getBuildIDHex()) {} + explicit FunctionBuildId(ContextPtr context) : FunctionConstantBase(SymbolIndex::instance()->getBuildIDHex(), context->isDistributed()) {} }; #endif @@ -35,7 +35,7 @@ namespace public: static constexpr auto name = "hostName"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionHostName(ContextPtr context) : FunctionConstantBase(context, DNSResolver::instance().getHostName()) {} + explicit FunctionHostName(ContextPtr context) : FunctionConstantBase(DNSResolver::instance().getHostName(), context->isDistributed()) {} }; @@ -44,7 +44,7 @@ namespace public: static constexpr auto name = "serverUUID"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionServerUUID(ContextPtr context) : FunctionConstantBase(context, ServerUUID::get()) {} + explicit FunctionServerUUID(ContextPtr context) : FunctionConstantBase(ServerUUID::get(), context->isDistributed()) {} }; @@ -53,7 +53,7 @@ namespace public: static constexpr auto name = "tcpPort"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTcpPort(ContextPtr context) : FunctionConstantBase(context, context->getTCPPort()) {} + explicit FunctionTcpPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {} }; @@ -63,7 +63,7 @@ namespace public: static constexpr auto name = "timezone"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(context, String{DateLUT::instance().getTimeZone()}) {} + explicit FunctionTimezone(ContextPtr context) : FunctionConstantBase(String{DateLUT::instance().getTimeZone()}, context->isDistributed()) {} }; @@ -73,7 +73,7 @@ namespace public: static constexpr auto name = "uptime"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionUptime(ContextPtr context) : FunctionConstantBase(context, context->getUptimeSeconds()) {} + explicit FunctionUptime(ContextPtr context) : FunctionConstantBase(context->getUptimeSeconds(), context->isDistributed()) {} }; @@ -83,14 +83,17 @@ namespace public: static constexpr auto name = "version"; static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } - explicit FunctionVersion(ContextPtr context) : FunctionConstantBase(context, VERSION_STRING) {} + explicit FunctionVersion(ContextPtr context) : FunctionConstantBase(VERSION_STRING, context->isDistributed()) {} }; class FunctionZooKeeperSessionUptime : public FunctionConstantBase { public: static constexpr auto name = "zookeeperSessionUptime"; - explicit FunctionZooKeeperSessionUptime(ContextPtr context) : FunctionConstantBase(context, context->getZooKeeperSessionUptime()) {} + explicit FunctionZooKeeperSessionUptime(ContextPtr context) + : FunctionConstantBase(context->getZooKeeperSessionUptime(), context->isDistributed()) + { + } static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; } From 5cc379392579978f5f398497c8a494508c2e40b3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 9 Oct 2021 11:50:06 +0800 Subject: [PATCH 319/950] Add shutdown_wait_unfinished_queries setting --- programs/server/Server.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index cd5d72cfba4..79a41078a77 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1550,7 +1550,8 @@ if (ThreadFuzzer::instance().isEffective()) LOG_INFO(log, "Closed all listening sockets."); /// Killing remaining queries. - global_context->getProcessList().killAllQueries(); + if (!config().getBool("shutdown_wait_unfinished_queries", false)) + global_context->getProcessList().killAllQueries(); if (current_connections) current_connections = waitServersToFinish(*servers, config().getInt("shutdown_wait_unfinished", 5)); From ddf9e524cc1012271f433cb526bc921acd7c5376 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 05:14:33 +0000 Subject: [PATCH 320/950] test poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 39fd359765a..c273106994a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 39fd359765a3a77b46d94ec3c5def3c7802a920f +Subproject commit c273106994a57b916a446de6afdc8489867f67fb From f7c1e25526eba1d7221f48eed9e6d8dc3263b7d8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 9 Oct 2021 09:00:51 +0300 Subject: [PATCH 321/950] Fix tidy --- src/Core/examples/coro.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp index e11d0af9e22..0f152d8090a 100644 --- a/src/Core/examples/coro.cpp +++ b/src/Core/examples/coro.cpp @@ -77,7 +77,7 @@ struct Task return *value; } - explicit Task(coro_handle handle) : my(handle), tag(handle.promise().tag) + Task(coro_handle handle) : my(handle), tag(handle.promise().tag) // NOLINT(google-explicit-constructor) { assert(handle); my.promise().r = this; From bed09ee68d01da683b685e8b5dbd93925e0c93fb Mon Sep 17 00:00:00 2001 From: vesslanjin Date: Sat, 9 Oct 2021 02:56:10 -0400 Subject: [PATCH 322/950] Whitespace fix Signed-off-by: vesslanjin --- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnsCommon.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index ea66ba73d8e..ab14ea3fb87 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -255,7 +255,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result else { size_t res_chars_size = res->chars.size(); - while(mask) + while (mask) { size_t index = __builtin_ctz(mask); res->chars.resize(res_chars_size + n); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index f45b0581213..85218402428 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -333,7 +333,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s } else { - while(mask) + while (mask) { size_t index = __builtin_ctz(mask); res_data.push_back(data_pos[index]); diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 8a6f5c24a32..d2f6883552a 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -258,7 +258,7 @@ namespace } else { - while(mask) + while (mask) { size_t index = __builtin_ctz(mask); copy_array(offsets_pos + index); From 939d38b13e20f0878f3a6f4c7d237ee97985945f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 9 Oct 2021 10:01:58 +0300 Subject: [PATCH 323/950] Fix gtest build. --- src/Storages/tests/gtest_row_source_bits_test.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/tests/gtest_row_source_bits_test.cpp b/src/Storages/tests/gtest_row_source_bits_test.cpp index 0e7db4b8ab8..c8ea48ff43c 100644 --- a/src/Storages/tests/gtest_row_source_bits_test.cpp +++ b/src/Storages/tests/gtest_row_source_bits_test.cpp @@ -1,6 +1,6 @@ #include -#include +#include using DB::RowSourcePart; From fa2b277e627873839b05f35b80790275e7abafa3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 07:57:37 +0000 Subject: [PATCH 324/950] improment format of AlterQuery --- src/Parsers/ASTAlterQuery.cpp | 170 +++++++++--------- .../02048_alter_command_format.reference | 5 + .../0_stateless/02048_alter_command_format.sh | 7 + 3 files changed, 94 insertions(+), 88 deletions(-) create mode 100644 tests/queries/0_stateless/02048_alter_command_format.reference create mode 100755 tests/queries/0_stateless/02048_alter_command_format.sh diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 06a0b6d46f5..ab64c319807 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -1,12 +1,11 @@ -#include -#include #include +#include +#include #include namespace DB { - namespace ErrorCodes { extern const int UNEXPECTED_AST_STRUCTURE; @@ -76,38 +75,37 @@ ASTPtr ASTAlterCommand::clone() const return res; } -void ASTAlterCommand::formatImpl( - const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' '); - if (type == ASTAlterCommand::ADD_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD COLUMN " << (if_not_exists ? "IF NOT EXISTS " : "") + << (settings.hilite ? hilite_none : ""); col_decl->formatImpl(settings, state, frame); if (first) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : ""); - else if (column) /// AFTER + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : ""); + else if (column) /// AFTER { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::DROP_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << (clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_column ? "CLEAR " : "DROP ") << "COLUMN " + << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::MODIFY_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") + << (settings.hilite ? hilite_none : ""); col_decl->formatImpl(settings, state, frame); if (!remove_property.empty()) @@ -117,152 +115,152 @@ void ASTAlterCommand::formatImpl( else { if (first) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : ""); - else if (column) /// AFTER + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : ""); + else if (column) /// AFTER { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); } } } else if (type == ASTAlterCommand::MATERIALIZE_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << "MATERIALIZE COLUMN " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE COLUMN " << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::COMMENT_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "COMMENT COLUMN " << (if_exists ? "IF EXISTS " : "") + << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); settings.ostr << " " << (settings.hilite ? hilite_none : ""); comment->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MODIFY_COMMENT) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COMMENT" << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY COMMENT" << (settings.hilite ? hilite_none : ""); settings.ostr << " " << (settings.hilite ? hilite_none : ""); comment->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MODIFY_ORDER_BY) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : ""); order_by->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MODIFY_SAMPLE_BY) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : ""); sample_by->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::ADD_INDEX) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "") + << (settings.hilite ? hilite_none : ""); index_decl->formatImpl(settings, state, frame); if (first) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : ""); - else if (index) /// AFTER + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : ""); + else if (index) /// AFTER { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : ""); index->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::DROP_INDEX) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << (clear_index ? "CLEAR " : "DROP ") << "INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_index ? "CLEAR " : "DROP ") << "INDEX " + << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); index->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::MATERIALIZE_INDEX) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : ""); index->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::ADD_CONSTRAINT) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD CONSTRAINT " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD CONSTRAINT " << (if_not_exists ? "IF NOT EXISTS " : "") + << (settings.hilite ? hilite_none : ""); constraint_decl->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::DROP_CONSTRAINT) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP CONSTRAINT " << (if_exists ? "IF EXISTS " : "") + << (settings.hilite ? hilite_none : ""); constraint->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::ADD_PROJECTION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD PROJECTION " << (if_not_exists ? "IF NOT EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD PROJECTION " << (if_not_exists ? "IF NOT EXISTS " : "") + << (settings.hilite ? hilite_none : ""); projection_decl->formatImpl(settings, state, frame); if (first) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FIRST " << (settings.hilite ? hilite_none : ""); else if (projection) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AFTER " << (settings.hilite ? hilite_none : ""); projection->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::DROP_PROJECTION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << (clear_projection ? "CLEAR " : "DROP ") << "PROJECTION " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << (clear_projection ? "CLEAR " : "DROP ") << "PROJECTION " + << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); projection->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::MATERIALIZE_PROJECTION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << "MATERIALIZE PROJECTION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE PROJECTION " << (settings.hilite ? hilite_none : ""); projection->formatImpl(settings, state, frame); if (partition) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } } else if (type == ASTAlterCommand::DROP_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str - << (detach ? "DETACH" : "DROP") << (part ? " PART " : " PARTITION ") + settings.ostr << (settings.hilite ? hilite_keyword : "") << (detach ? "DETACH" : "DROP") << (part ? " PART " : " PARTITION ") << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::DROP_DETACHED_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP DETACHED" << (part ? " PART " : " PARTITION ") + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP DETACHED" << (part ? " PART " : " PARTITION ") << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::ATTACH_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ATTACH " - << (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH " << (part ? "PART " : "PARTITION ") + << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MOVE_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MOVE " - << (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MOVE " << (part ? "PART " : "PARTITION ") + << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); settings.ostr << " TO "; switch (move_destination_type) @@ -280,8 +278,7 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_database) << (settings.hilite ? hilite_none : "") << "."; } - settings.ostr << (settings.hilite ? hilite_identifier : "") - << backQuoteIfNeed(to_table) + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(to_table) << (settings.hilite ? hilite_none : ""); return; default: @@ -294,7 +291,7 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::REPLACE_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << (replace ? "REPLACE" : "ATTACH") << " PARTITION " + settings.ostr << (settings.hilite ? hilite_keyword : "") << (replace ? "REPLACE" : "ATTACH") << " PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : ""); @@ -307,57 +304,56 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::FETCH_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FETCH " - << (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "FETCH " << (part ? "PART " : "PARTITION ") + << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); - settings.ostr << (settings.hilite ? hilite_keyword : "") - << " FROM " << (settings.hilite ? hilite_none : "") << DB::quote << from; + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "") << DB::quote << from; } else if (type == ASTAlterCommand::FREEZE_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FREEZE PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "FREEZE PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); if (!with_name.empty()) { - settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") - << " " << DB::quote << with_name; + settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " " + << DB::quote << with_name; } } else if (type == ASTAlterCommand::FREEZE_ALL) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "FREEZE"; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "FREEZE"; if (!with_name.empty()) { - settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") - << " " << DB::quote << with_name; + settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " " + << DB::quote << with_name; } } else if (type == ASTAlterCommand::UNFREEZE_PARTITION) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "UNFREEZE PARTITION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "UNFREEZE PARTITION " << (settings.hilite ? hilite_none : ""); partition->formatImpl(settings, state, frame); if (!with_name.empty()) { - settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") - << " " << DB::quote << with_name; + settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " " + << DB::quote << with_name; } } else if (type == ASTAlterCommand::UNFREEZE_ALL) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "UNFREEZE"; + settings.ostr << (settings.hilite ? hilite_keyword : "") << "UNFREEZE"; if (!with_name.empty()) { - settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") - << " " << DB::quote << with_name; + settings.ostr << " " << (settings.hilite ? hilite_keyword : "") << "WITH NAME" << (settings.hilite ? hilite_none : "") << " " + << DB::quote << with_name; } } else if (type == ASTAlterCommand::DELETE) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DELETE" << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DELETE" << (settings.hilite ? hilite_none : ""); if (partition) { @@ -370,7 +366,7 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::UPDATE) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "UPDATE " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "UPDATE " << (settings.hilite ? hilite_none : ""); update_assignments->formatImpl(settings, state, frame); if (partition) @@ -384,17 +380,16 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::MODIFY_TTL) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY TTL " << (settings.hilite ? hilite_none : ""); ttl->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::REMOVE_TTL) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REMOVE TTL" << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE TTL" << (settings.hilite ? hilite_none : ""); } else if (type == ASTAlterCommand::MATERIALIZE_TTL) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MATERIALIZE TTL" - << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MATERIALIZE TTL" << (settings.hilite ? hilite_none : ""); if (partition) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " IN PARTITION " << (settings.hilite ? hilite_none : ""); @@ -403,31 +398,32 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::MODIFY_SETTING) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY SETTING " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SETTING " << (settings.hilite ? hilite_none : ""); settings_changes->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::RESET_SETTING) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RESET SETTING " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "RESET SETTING " << (settings.hilite ? hilite_none : ""); settings_resets->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MODIFY_DATABASE_SETTING) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY SETTING " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SETTING " << (settings.hilite ? hilite_none : ""); settings_changes->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::MODIFY_QUERY) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY QUERY " << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY QUERY " << settings.nl_or_ws + << (settings.hilite ? hilite_none : ""); select->formatImpl(settings, state, frame); } else if (type == ASTAlterCommand::LIVE_VIEW_REFRESH) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REFRESH " << (settings.hilite ? hilite_none : ""); } else if (type == ASTAlterCommand::RENAME_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RENAME COLUMN " << (if_exists ? "IF EXISTS " : "") + settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); @@ -522,12 +518,10 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState } formatOnCluster(settings); - settings.ostr << settings.nl_or_ws; FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; - ++frame_nested.indent; - static_cast(command_list)->formatImpl(settings, state, frame_nested); + static_cast(command_list)->formatImplMultiline(settings, state, frame_nested); } } diff --git a/tests/queries/0_stateless/02048_alter_command_format.reference b/tests/queries/0_stateless/02048_alter_command_format.reference new file mode 100644 index 00000000000..719c9391f46 --- /dev/null +++ b/tests/queries/0_stateless/02048_alter_command_format.reference @@ -0,0 +1,5 @@ +ALTER TABLE t + DROP COLUMN a, + DROP COLUMN b, + DROP COLUMN c, + ADD COLUMN `d` UInt8 diff --git a/tests/queries/0_stateless/02048_alter_command_format.sh b/tests/queries/0_stateless/02048_alter_command_format.sh new file mode 100755 index 00000000000..2f088cf8176 --- /dev/null +++ b/tests/queries/0_stateless/02048_alter_command_format.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "alter table t drop column a, drop column b, drop column c, add column d UInt8" | $CLICKHOUSE_FORMAT; From 2204597cfef8767472f2dd25a6e6972fd45845c7 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sat, 9 Oct 2021 16:49:49 +0800 Subject: [PATCH 325/950] add FunctionOSKernelVersion --- .../registerFunctionsMiscellaneous.cpp | 2 ++ src/Functions/serverConstants.cpp | 24 ++++++++++++++ src/Interpreters/getOSKernelVersion.cpp | 31 ------------------- src/Interpreters/getOSKernelVersion.h | 21 ------------- 4 files changed, 26 insertions(+), 52 deletions(-) delete mode 100644 src/Interpreters/getOSKernelVersion.cpp delete mode 100644 src/Interpreters/getOSKernelVersion.h diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index dfd986c5f82..8fc084b3d1a 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -81,6 +81,7 @@ void registerFunctionQueryID(FunctionFactory & factory); void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); +void registerFunctionOSKernelVersion(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -162,6 +163,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionInitialQueryID(factory); registerFunctionServerUUID(factory); registerFunctionZooKeeperSessionUptime(factory); + registerFunctionOSKernelVersion(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 9a53a5cf582..170f4cb86ee 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -7,6 +7,10 @@ #include #include +#if defined(OS_LINUX) +# include +#endif + #if !defined(ARCADIA_BUILD) # include #endif @@ -93,6 +97,17 @@ namespace explicit FunctionZooKeeperSessionUptime(ContextPtr context) : FunctionConstantBase(context, context->getZooKeeperSessionUptime()) {} static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; + +#if defined(OS_LINUX) + class FunctionOSKernelVersion : public FunctionConstantBase + { + public: + static constexpr auto name = "OSKernelVersion"; + explicit FunctionOSKernelVersion(ContextPtr context) : FunctionConstantBase(context, Poco::Environment::osName() + " " + Poco::Environment::osVersion()) {} + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + }; +#endif + } @@ -140,5 +155,14 @@ void registerFunctionZooKeeperSessionUptime(FunctionFactory & factory) factory.registerFunction(); } + +void registerFunctionOSKernelVersion(FunctionFactory & factory) +{ +#if defined(OS_LINUX) + factory.registerFunction(); +#endif +} + + } diff --git a/src/Interpreters/getOSKernelVersion.cpp b/src/Interpreters/getOSKernelVersion.cpp deleted file mode 100644 index c4b4564f46e..00000000000 --- a/src/Interpreters/getOSKernelVersion.cpp +++ /dev/null @@ -1,31 +0,0 @@ -#if defined(OS_LINUX) -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYSTEM_ERROR; -} - -String getOSKernelVersion() -{ - struct utsname os_kernel_info; - int buf = uname(&os_kernel_info); - if (buf < 0) - { - throw Exception( - "EFAULT buffer is not valid.", - ErrorCodes::SYSTEM_ERROR); - } - else - { - return String(os_kernel_info.sysname) + " " + String(os_kernel_info.release); - } -} - -} - -#endif \ No newline at end of file diff --git a/src/Interpreters/getOSKernelVersion.h b/src/Interpreters/getOSKernelVersion.h deleted file mode 100644 index fc3c7583aef..00000000000 --- a/src/Interpreters/getOSKernelVersion.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once -#if defined(OS_LINUX) - -#include - -#include -#include - -namespace DB -{ - -/// Returns String with OS Kernel version. -/* To get name and information about current kernel. - For simplicity, the function can be implemented only for Linux. -*/ - -String getOSKernelVersion(); - -} - -#endif \ No newline at end of file From 67ff0f5dba5b95ec67c6e79f721fd5904ebb4115 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Fri, 8 Oct 2021 13:07:11 +0300 Subject: [PATCH 326/950] Refactoring: moved `comment` out of ASTStorage to ASTCreateQuery --- src/Databases/DatabaseFactory.cpp | 4 +- src/Databases/DatabaseMemory.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 16 +++---- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/SQLite/DatabaseSQLite.cpp | 2 +- src/Parsers/ASTCreateQuery.cpp | 24 +++++++---- src/Parsers/ASTCreateQuery.h | 2 +- src/Parsers/ParserCreateQuery.cpp | 42 +++++++++++++------ src/Storages/StorageFactory.cpp | 4 +- 10 files changed, 59 insertions(+), 41 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 9cf600d5cdf..931c5188159 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -79,8 +79,8 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName()); // Attach database metadata - if (impl && create.storage && create.storage->comment) - impl->setDatabaseComment(create.storage->comment->as()->value.safeGet()); + if (impl && create.comment) + impl->setDatabaseComment(create.comment->as()->value.safeGet()); return impl; } diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 8f909d280c5..8b6aca4152d 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -71,7 +71,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const create_query->storage->set(create_query->storage->engine, makeASTFunction(getEngineName())); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->storage->set(create_query->storage->comment, std::make_shared(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index d681d4f83da..77518d76767 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -188,12 +188,12 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo if (metadata.settings_changes) storage_ast.set(storage_ast.settings, metadata.settings_changes); } - - if (metadata.comment.empty()) - storage_ast.reset(storage_ast.comment); - else - storage_ast.set(storage_ast.comment, std::make_shared(metadata.comment)); } + + if (metadata.comment.empty()) + ast_create_query.reset(ast_create_query.comment); + else + ast_create_query.set(ast_create_query.comment, std::make_shared(metadata.comment)); } @@ -531,11 +531,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const if (const auto database_comment = getDatabaseComment(); !database_comment.empty()) { auto & ast_create_query = ast->as(); - // TODO(nemkov): this is a precaution and should never happen, remove if there are no failed tests on CI/CD. - if (!ast_create_query.storage) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ASTCreateQuery lacks engine clause, but a comment is present."); - - ast_create_query.storage->set(ast_create_query.storage->comment, std::make_shared(database_comment)); + ast_create_query.set(ast_create_query.comment, std::make_shared(database_comment)); } return ast; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index a30341ae927..2377b7da809 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -198,7 +198,7 @@ ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->storage->set(create_query->storage->comment, std::make_shared(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 98f868dadc2..430caa0f4a3 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -349,7 +349,7 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->storage->set(create_query->storage->comment, std::make_shared(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index 7d7cd523179..05bc2f24834 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -164,7 +164,7 @@ ASTPtr DatabaseSQLite::getCreateDatabaseQuery() const create_query->set(create_query->storage, database_engine_define); if (const auto comment_value = getDatabaseComment(); !comment_value.empty()) - create_query->storage->set(create_query->storage->comment, std::make_shared(comment_value)); + create_query->set(create_query->comment, std::make_shared(comment_value)); return create_query; } diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index be973a988bd..2c048886247 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -32,9 +32,6 @@ ASTPtr ASTStorage::clone() const if (settings) res->set(res->settings, settings->clone()); - if (comment) - res->set(res->comment, comment->clone()); - return res; } @@ -75,12 +72,6 @@ void ASTStorage::formatImpl(const FormatSettings & s, FormatState & state, Forma s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "SETTINGS " << (s.hilite ? hilite_none : ""); settings->formatImpl(s, state, frame); } - if (comment) - { - s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << "COMMENT " << (s.hilite ? hilite_none : ""); - comment->formatImpl(s, state, frame); - } - } @@ -217,6 +208,9 @@ ASTPtr ASTCreateQuery::clone() const res->set(res->dictionary, dictionary->clone()); } + if (comment) + res->set(res->comment, comment->clone()); + cloneOutputOptions(*res); return res; @@ -245,6 +239,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (storage) storage->formatImpl(settings, state, frame); + if (comment) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : ""); + comment->formatImpl(settings, state, frame); + } + return; } @@ -405,6 +405,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH " << (settings.hilite ? hilite_none : ""); tables->formatImpl(settings, state, frame); } + + if (comment) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << settings.nl_or_ws << "COMMENT " << (settings.hilite ? hilite_none : ""); + comment->formatImpl(settings, state, frame); + } } } diff --git a/src/Parsers/ASTCreateQuery.h b/src/Parsers/ASTCreateQuery.h index e4f2b628886..51c4ac15e67 100644 --- a/src/Parsers/ASTCreateQuery.h +++ b/src/Parsers/ASTCreateQuery.h @@ -23,7 +23,6 @@ public: IAST * order_by = nullptr; IAST * sample_by = nullptr; IAST * ttl_table = nullptr; - IAST * comment = nullptr; ASTSetQuery * settings = nullptr; @@ -75,6 +74,7 @@ public: String as_table; ASTPtr as_table_function; ASTSelectWithUnionQuery * select = nullptr; + IAST * comment = nullptr; bool is_dictionary{false}; /// CREATE DICTIONARY ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index d4525883e36..07e321c3376 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -26,6 +26,20 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ +ASTPtr parseComment(IParser::Pos & pos, Expected & expected) +{ + ParserKeyword s_comment("COMMENT"); + ParserStringLiteral string_literal_parser; + ASTPtr comment; + + s_comment.ignore(pos, expected) && string_literal_parser.parse(pos, comment, expected); + + return comment; +} +} + bool ParserNestedTable::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken open(TokenType::OpeningRoundBracket); @@ -314,7 +328,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_sample_by("SAMPLE BY"); ParserKeyword s_ttl("TTL"); ParserKeyword s_settings("SETTINGS"); - ParserKeyword s_comment("COMMENT"); ParserIdentifierWithOptionalParameters ident_with_optional_params_p; ParserExpression expression_p; @@ -329,7 +342,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr sample_by; ASTPtr ttl_table; ASTPtr settings; - ASTPtr comment_expression; if (!s_engine.ignore(pos, expected)) return false; @@ -387,13 +399,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - if (s_comment.ignore(pos, expected)) - { - /// should be followed by a string literal - if (!string_literal_parser.parse(pos, comment_expression, expected)) - return false; - } - break; } @@ -407,8 +412,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) storage->set(storage->settings, settings); - storage->set(storage->comment, comment_expression); - node = storage; return true; } @@ -568,7 +571,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe } } } - + auto comment = parseComment(pos, expected); auto query = std::make_shared(); node = query; @@ -590,6 +593,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe query->set(query->columns_list, columns_list); query->set(query->storage, storage); + if (comment) + query->set(query->comment, comment); + if (query->storage && query->columns_list && query->columns_list->primary_key) { if (query->storage->primary_key) @@ -803,7 +809,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e } storage_p.parse(pos, storage, expected); - + auto comment = parseComment(pos, expected); auto query = std::make_shared(); node = query; @@ -816,6 +822,8 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e query->cluster = cluster_str; query->set(query->storage, storage); + if (comment) + query->set(query->comment, comment); return true; } @@ -934,6 +942,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (!select_p.parse(pos, select, expected)) return false; + auto comment = parseComment(pos, expected); auto query = std::make_shared(); node = query; @@ -958,6 +967,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->set(query->columns_list, columns_list); query->set(query->storage, storage); + if (comment) + query->set(query->comment, comment); tryGetIdentifierNameInto(as_database, query->as_database); tryGetIdentifierNameInto(as_table, query->as_table); @@ -1039,6 +1050,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E return false; } + auto comment = parseComment(pos, expected); + auto query = std::make_shared(); node = query; query->is_dictionary = true; @@ -1056,6 +1069,9 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E query->set(query->dictionary, dictionary); query->cluster = cluster_str; + if (comment) + query->set(query->comment, comment); + return true; } diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index cfa50b95487..ba2ac3f72a3 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -146,8 +146,8 @@ StoragePtr StorageFactory::get( throw Exception("Unknown table engine " + name, ErrorCodes::UNKNOWN_STORAGE); } - if (storage_def->comment) - comment = storage_def->comment->as().value.get(); + if (query.comment) + comment = query.comment->as().value.get(); auto check_feature = [&](String feature_description, FeatureMatcherFn feature_matcher_fn) { From 199eda4ee37a9a5f60219145ec3f427f985679b7 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Fri, 8 Oct 2021 11:48:10 +0300 Subject: [PATCH 327/950] Test for dictionaries with comments --- ...4_create_dictionary_with_comment.reference | 0 .../02024_create_dictionary_with_comment.sql | 32 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02024_create_dictionary_with_comment.reference create mode 100644 tests/queries/0_stateless/02024_create_dictionary_with_comment.sql diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference b/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql new file mode 100644 index 00000000000..de71d0d976b --- /dev/null +++ b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql @@ -0,0 +1,32 @@ +----------------------------------------------------------------------------------- +-- Check that `DICTIONARY` can be created with a `COMMENT` clause +-- and comment is visible both in `comment` column of `system.dictionaries` +-- and `SHOW CREATE DICTIONARY`. +----------------------------------------------------------------------------------- + +-- prerequisites +CREATE TABLE source_table +( + id UInt64, + value String +) ENGINE = Memory(); + +INSERT INTO source_table VALUES (1, 'First'); +INSERT INTO source_table VALUES (2, 'Second'); + +DROP DICTIONARY IF EXISTS 2024_dictionary_with_comment; + +CREATE DICTIONARY 2024_dictionary_with_comment +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) +LAYOUT(FLAT()) +LIFETIME(MIN 0 MAX 1000) +COMMENT 'Test dictionary with comment'; + +SHOW CREATE DICTIONARY 2024_dictionary_with_comment; + +SELECT comment FROM system.dictionaries WHERE name == '2024_dictionary_with_comment'; \ No newline at end of file From fecfcf9b66afa96a3107c5b2debe420c5c894161 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sat, 9 Oct 2021 13:17:47 +0300 Subject: [PATCH 328/950] Update metrica.md --- docs/ru/getting-started/example-datasets/metrica.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md index 15c0cc14517..ee764ff4879 100644 --- a/docs/ru/getting-started/example-datasets/metrica.md +++ b/docs/ru/getting-started/example-datasets/metrica.md @@ -37,7 +37,7 @@ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv # создадим таблицу hits_v1 clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" +clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # создадим таблицу hits_100m_obfuscated clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" From 3940382c2f4e84c17eb7b9ff8819eca1a1bc579d Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Sat, 9 Oct 2021 13:34:02 +0300 Subject: [PATCH 329/950] Remove defined(__POPCNT__) from column filter --- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnsCommon.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index ab14ea3fb87..733ecaa979a 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -231,7 +231,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result const UInt8 * filt_end = filt_pos + col_size; const UInt8 * data_pos = chars.data(); -#if defined(__SSE2__) && defined(__POPCNT__) +#ifdef __SSE2__ /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 85218402428..a769cd93037 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -311,7 +311,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); -#if defined(__SSE2__) && defined(__POPCNT__) +#ifdef __SSE2__ /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index d2f6883552a..a4d7de34382 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -229,7 +229,7 @@ namespace memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T)); }; - #if defined(__SSE2__) && defined(__POPCNT__) + #ifdef __SSE2__ const __m128i zero_vec = _mm_setzero_si128(); static constexpr size_t SIMD_BYTES = 16; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From aae07e1947c27412ac9ea13c29cfd58d48fb8153 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 10:34:57 +0000 Subject: [PATCH 330/950] run test again --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index c273106994a..5e822a5015a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit c273106994a57b916a446de6afdc8489867f67fb +Subproject commit 5e822a5015a431a288c4b364fd82511d95731f78 From 0e8d2f33d76d0644c9aa673a76248c7be67f76d5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 10:51:10 +0000 Subject: [PATCH 331/950] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 5e822a5015a..ab0b711c68d 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 5e822a5015a431a288c4b364fd82511d95731f78 +Subproject commit ab0b711c68d27feabc2e82c840bd5225b185ef4a From 96a9d99ab658f3e0e0dc4c21bf6f01ebcbe41dbb Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sat, 9 Oct 2021 19:53:21 +0800 Subject: [PATCH 332/950] add test --- src/Functions/registerFunctionsMiscellaneous.cpp | 4 ++-- src/Functions/serverConstants.cpp | 12 ++++++------ .../02095_function_get_os_kernel_version.reference | 1 + .../02095_function_get_os_kernel_version.sql | 1 + 4 files changed, 10 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02095_function_get_os_kernel_version.reference create mode 100644 tests/queries/0_stateless/02095_function_get_os_kernel_version.sql diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 8fc084b3d1a..dc062ab148a 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -81,7 +81,7 @@ void registerFunctionQueryID(FunctionFactory & factory); void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionServerUUID(FunctionFactory &); void registerFunctionZooKeeperSessionUptime(FunctionFactory &); -void registerFunctionOSKernelVersion(FunctionFactory &); +void registerFunctionGetOSKernelVersion(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -163,7 +163,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionInitialQueryID(factory); registerFunctionServerUUID(factory); registerFunctionZooKeeperSessionUptime(factory); - registerFunctionOSKernelVersion(factory); + registerFunctionGetOSKernelVersion(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 170f4cb86ee..f92bd6f6653 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -99,12 +99,12 @@ namespace }; #if defined(OS_LINUX) - class FunctionOSKernelVersion : public FunctionConstantBase + class FunctionGetOSKernelVersion : public FunctionConstantBase { public: - static constexpr auto name = "OSKernelVersion"; - explicit FunctionOSKernelVersion(ContextPtr context) : FunctionConstantBase(context, Poco::Environment::osName() + " " + Poco::Environment::osVersion()) {} - static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + static constexpr auto name = "getOSKernelVersion"; + explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionConstantBase(context, Poco::Environment::osName() + " " + Poco::Environment::osVersion()) {} + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; #endif @@ -156,10 +156,10 @@ void registerFunctionZooKeeperSessionUptime(FunctionFactory & factory) } -void registerFunctionOSKernelVersion(FunctionFactory & factory) +void registerFunctionGetOSKernelVersion([[maybe_unused]] FunctionFactory & factory) { #if defined(OS_LINUX) - factory.registerFunction(); + factory.registerFunction(); #endif } diff --git a/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference b/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference new file mode 100644 index 00000000000..9ec3c4aef9b --- /dev/null +++ b/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference @@ -0,0 +1 @@ +Linux 0 diff --git a/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql b/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql new file mode 100644 index 00000000000..01cd1057d84 --- /dev/null +++ b/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql @@ -0,0 +1 @@ +WITH splitByChar(' ', getOSKernelVersion()) AS version_pair SELECT version_pair[1], toUInt32(splitByChar('.', version_pair[2])[1]) > 3 \ No newline at end of file From b8bde2d4513a5f9f0f8f2ffc563c2eaf6d591e1a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Oct 2021 15:05:21 +0300 Subject: [PATCH 333/950] Fix test_backup_restore after #29649 CI: https://clickhouse-test-reports.s3.yandex.net/29856/42ca2b4bb241827edf69bbd6938d6b19c31935f1/integration_tests_(asan).html#fail1 Cc: @CurtizJ --- tests/integration/test_backup_restore/test.py | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_backup_restore/test.py b/tests/integration/test_backup_restore/test.py index a093b06cd5c..b990cec2364 100644 --- a/tests/integration/test_backup_restore/test.py +++ b/tests/integration/test_backup_restore/test.py @@ -39,12 +39,10 @@ def get_last_backup_path(instance, database, table): return os.path.join(path_to_data, 'shadow', increment, 'data', database, table) def copy_backup_to_detached(instance, database, src_table, dst_table): - fp_increment = os.path.join(path_to_data, 'shadow/increment.txt') - increment = instance.exec_in_container(['cat', fp_increment]).strip() - fp_backup = os.path.join(path_to_data, 'shadow', increment, 'data', database, src_table) + fp_backup = os.path.join(path_to_data, 'shadow', '*', 'data', database, src_table) fp_detached = os.path.join(path_to_data, 'data', database, dst_table, 'detached') - logging.debug(f'copy from {fp_backup} to {fp_detached}. increment {fp_increment}') - instance.exec_in_container(['cp', '-r', f'{fp_backup}', '-T' , f'{fp_detached}']) + logging.debug(f'copy from {fp_backup} to {fp_detached}') + instance.exec_in_container(['bash', '-c', f'cp -r {fp_backup} -T {fp_detached}']) def test_restore(started_cluster): instance.query("CREATE TABLE test.tbl1 AS test.tbl") From fe12404cb017e2f41ae9d46c9d679cde765156ba Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Oct 2021 15:14:23 +0300 Subject: [PATCH 334/950] Fix tests test_log_family_s3, test_log_family_hdfs. --- tests/integration/test_log_family_hdfs/test.py | 14 ++++++++++++-- tests/integration/test_log_family_s3/test.py | 14 ++++++++++++-- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_log_family_hdfs/test.py b/tests/integration/test_log_family_hdfs/test.py index a38b067358e..7bb9cdfeaf5 100644 --- a/tests/integration/test_log_family_hdfs/test.py +++ b/tests/integration/test_log_family_hdfs/test.py @@ -31,10 +31,20 @@ def assert_objects_count(started_cluster, objects_count, path='data/'): hdfs_objects = fs.listdir('/clickhouse') assert objects_count == len(hdfs_objects) - +# TinyLog: files: id.bin, sizes.json +# INSERT overwrites 1 file (`sizes.json`) and appends 1 file (`id.bin`), so +# files_overhead=1, files_overhead_per_insert=1 +# +# Log: files: id.bin, __marks.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`id.bin`, `__marks.mrk`), so +# files_overhead=1, files_overhead_per_insert=2 +# +# StripeLog: files: data.bin, index.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`index.mrk`, `data.bin`), so +# files_overhead=1, files_overhead_per_insert=2 @pytest.mark.parametrize( "log_engine,files_overhead,files_overhead_per_insert", - [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) + [("TinyLog", 1, 1), ("Log", 1, 2), ("StripeLog", 1, 2)]) def test_log_family_hdfs(started_cluster, log_engine, files_overhead, files_overhead_per_insert): node = started_cluster.instances["node"] diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index 71d47a8a2e8..8531edd635f 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -30,10 +30,20 @@ def assert_objects_count(cluster, objects_count, path='data/'): logging.info("Existing S3 object: %s", str(object_meta)) assert objects_count == len(s3_objects) - +# TinyLog: files: id.bin, sizes.json +# INSERT overwrites 1 file (`sizes.json`) and appends 1 file (`id.bin`), so +# files_overhead=1, files_overhead_per_insert=1 +# +# Log: files: id.bin, __marks.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`id.bin`, `__marks.mrk`), so +# files_overhead=1, files_overhead_per_insert=2 +# +# StripeLog: files: data.bin, index.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`index.mrk`, `data.bin`), so +# files_overhead=1, files_overhead_per_insert=2 @pytest.mark.parametrize( "log_engine,files_overhead,files_overhead_per_insert", - [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) + [("TinyLog", 1, 1), ("Log", 1, 2), ("StripeLog", 1, 2)]) def test_log_family_s3(cluster, log_engine, files_overhead, files_overhead_per_insert): node = cluster.instances["node"] From 473f7bee3ad023b21de225c280b0d8e62979fdcc Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Sat, 9 Oct 2021 12:16:37 +0000 Subject: [PATCH 335/950] refactor --- programs/server/users.xml | 4 ++-- src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp | 3 +++ src/Server/HTTP/WriteBufferFromHTTPServerResponse.h | 8 ++++++++ src/Server/HTTPHandler.cpp | 9 ++++++--- tests/integration/test_settings_profile/test.py | 6 +++--- 5 files changed, 22 insertions(+), 8 deletions(-) diff --git a/programs/server/users.xml b/programs/server/users.xml index b935937a1e5..4cd0c88a0a2 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -19,8 +19,8 @@ --> random - - 1 + + diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp index 9131413a887..cba0b6f0592 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp @@ -29,6 +29,9 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders() { headers_started_sending = true; + if (add_cors_header) + response.set("Access-Control-Allow-Origin", "*"); + setResponseDefaultHeaders(response, keep_alive_timeout); if (!is_http_method_head) diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h index 7ce99b16261..665c2daebbd 100644 --- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h +++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h @@ -36,6 +36,7 @@ private: HTTPServerResponse & response; bool is_http_method_head; + bool add_cors_header = false; unsigned keep_alive_timeout = 0; bool compress = false; CompressionMethod compression_method; @@ -103,6 +104,13 @@ public: compression_level = level; } + /// Turn CORS on or off. + /// The setting has any effect only if HTTP headers haven't been sent yet. + void addHeaderCORS(bool enable_cors) + { + add_cors_header = enable_cors; + } + /// Don't send HTTP headers with progress more frequently. void setSendProgressInterval(size_t send_progress_interval_ms_) { diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 9edef8a7223..c50d0e753ed 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -745,10 +745,13 @@ void HTTPHandler::processQuery( if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress) static_cast(*in_post_maybe_compressed).disableChecksumming(); - /// Add CORS header if 'add_http_cors_header' setting is turned on or config has http_options_response, - /// which means that there are some headers to be sent, and the client passed Origin header. - if (settings.add_http_cors_header && config.has("http_options_response") && !request.get("Origin", "").empty()) + /// Add CORS header if 'add_http_cors_header' setting is turned on send * in Access-Control-Allow-Origin, + /// or if config has http_options_response, which means that there + /// are some headers to be sent, and the client passed Origin header. + if (config.has("http_options_response") && !request.get("Origin", "").empty()) tryAddHeadersFromConfig(response, config); + else if (settings.add_http_cors_header) + used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); auto append_callback = [context = context] (ProgressCallback callback) { diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index 048d09daaa5..7be0b395764 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -201,13 +201,13 @@ def test_show_profiles(): assert instance.query("SHOW CREATE PROFILE xyz") == "CREATE SETTINGS PROFILE xyz\n" assert instance.query( - "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" + "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" assert instance.query( - "SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" \ + "SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" \ "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" \ "CREATE SETTINGS PROFILE xyz\n" - expected_access = "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n" \ + expected_access = "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" \ "CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" \ "CREATE SETTINGS PROFILE xyz\n" assert expected_access in instance.query("SHOW ACCESS") From 12f59f2dcac5ec74e40724c612e95f4093cadbc5 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Sat, 9 Oct 2021 12:52:20 +0000 Subject: [PATCH 336/950] refactor --- programs/server/users.xml | 5 +---- src/Server/HTTPHandler.cpp | 6 +++--- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/programs/server/users.xml b/programs/server/users.xml index 4cd0c88a0a2..847e7768ed0 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -18,10 +18,7 @@ first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors. --> random - - - - + diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c50d0e753ed..42ff154e807 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -746,12 +746,12 @@ void HTTPHandler::processQuery( static_cast(*in_post_maybe_compressed).disableChecksumming(); /// Add CORS header if 'add_http_cors_header' setting is turned on send * in Access-Control-Allow-Origin, - /// or if config has http_options_response, which means that there + /// or if config has http_options_response, which means that there /// are some headers to be sent, and the client passed Origin header. if (config.has("http_options_response") && !request.get("Origin", "").empty()) tryAddHeadersFromConfig(response, config); - else if (settings.add_http_cors_header) - used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty()); + else if (settings.add_http_cors_header && !request.get("Origin", "").empty()) + used_output.out->addHeaderCORS(true); auto append_callback = [context = context] (ProgressCallback callback) { From a853cd57ca67014bcd5e73145aa1daf14f274b2f Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Sat, 9 Oct 2021 12:56:00 +0000 Subject: [PATCH 337/950] refactoring --- src/Server/HTTPHandler.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 42ff154e807..b7e7ac2ac33 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -748,10 +748,13 @@ void HTTPHandler::processQuery( /// Add CORS header if 'add_http_cors_header' setting is turned on send * in Access-Control-Allow-Origin, /// or if config has http_options_response, which means that there /// are some headers to be sent, and the client passed Origin header. - if (config.has("http_options_response") && !request.get("Origin", "").empty()) - tryAddHeadersFromConfig(response, config); - else if (settings.add_http_cors_header && !request.get("Origin", "").empty()) - used_output.out->addHeaderCORS(true); + if (!request.get("Origin", "").empty()) + { + if (config.has("http_options_response")) + tryAddHeadersFromConfig(response, config); + else if (settings.add_http_cors_header) + used_output.out->addHeaderCORS(true); + } auto append_callback = [context = context] (ProgressCallback callback) { From 98ecd59ebe5ebbc25884c6a577b85d13219039ec Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Sat, 9 Oct 2021 16:00:37 +0300 Subject: [PATCH 338/950] Update users.xml --- programs/server/users.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/users.xml b/programs/server/users.xml index 847e7768ed0..0f7dfadb8ea 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -18,7 +18,7 @@ first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors. --> random - + From 7753cc9c2843f737e8a946e2866626d724009477 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Sat, 9 Oct 2021 16:01:01 +0300 Subject: [PATCH 339/950] Update users.xml --- programs/server/users.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/users.xml b/programs/server/users.xml index 0f7dfadb8ea..fd5fe414579 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -18,7 +18,7 @@ first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors. --> random - + From c66942bba5eff720ad1e336bfde6bd676266f1df Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Oct 2021 16:15:31 +0300 Subject: [PATCH 340/950] Fix test_input_format_parallel_parsing_memory_tracking::test_memory_tracking_total Server has pretty low memory limit 3GB, so let's use local to generate the data. --- .../test_input_format_parallel_parsing_memory_tracking/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py index e8866d3a235..bc7f32bf544 100644 --- a/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py +++ b/tests/integration/test_input_format_parallel_parsing_memory_tracking/test.py @@ -30,7 +30,7 @@ def test_memory_tracking_total(): CREATE TABLE null (row String) ENGINE=Null; ''') instance.exec_in_container(['bash', '-c', - 'clickhouse client -q "SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), \' \') from numbers(10000)" > data.json']) + 'clickhouse local -q "SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), \' \') from numbers(10000)" > data.json']) for it in range(0, 20): # the problem can be triggered only via HTTP, # since clickhouse-client parses the data by itself. From 24568c9de5dc629fd1ec7054566a48530a6ce777 Mon Sep 17 00:00:00 2001 From: Artur <613623@mail.ru> Date: Sat, 9 Oct 2021 13:55:30 +0000 Subject: [PATCH 341/950] update test reference --- tests/queries/0_stateless/00372_cors_header.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00372_cors_header.reference b/tests/queries/0_stateless/00372_cors_header.reference index e22493782f0..2f1465d1598 100644 --- a/tests/queries/0_stateless/00372_cors_header.reference +++ b/tests/queries/0_stateless/00372_cors_header.reference @@ -1,3 +1,3 @@ 1 +1 0 -0 From 764b17787b7c260b71662f703b90094d130063c6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 14:25:04 +0000 Subject: [PATCH 342/950] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index ab0b711c68d..237b0184424 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit ab0b711c68d27feabc2e82c840bd5225b185ef4a +Subproject commit 237b018442435f732911e9fc6dd1d4d5328968a5 From f25fbe37403355cb16e0b0c39689f1f1bb3ec50f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Oct 2021 18:37:38 +0300 Subject: [PATCH 343/950] Update CCTZ --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index c0f1bcb97fd..9edd0861d83 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit c0f1bcb97fd2782f7c3f972fadd5aad5affac4b8 +Subproject commit 9edd0861d8328b2ae77e8fb5f4d7dcd1cf33b42b From f0b7e54e83d4435734eecb56b3881d30e1d5d96c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 9 Oct 2021 18:42:53 +0300 Subject: [PATCH 344/950] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d203fcba4c8..718aa751cc2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -### ClickHouse release v21.10, 2021-10-08 +### ClickHouse release v21.10, 2021-10-14 #### Backward Incompatible Change From bc093e2b23bd64e34042374d13a4a3cef7939c29 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sun, 10 Oct 2021 00:06:11 +0800 Subject: [PATCH 345/950] fix test --- .../0_stateless/02095_function_get_os_kernel_version.reference | 2 +- .../0_stateless/02095_function_get_os_kernel_version.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference b/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference index 9ec3c4aef9b..9b075671eac 100644 --- a/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference +++ b/tests/queries/0_stateless/02095_function_get_os_kernel_version.reference @@ -1 +1 @@ -Linux 0 +Linux diff --git a/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql b/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql index 01cd1057d84..ed38abb64a9 100644 --- a/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql +++ b/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql @@ -1 +1 @@ -WITH splitByChar(' ', getOSKernelVersion()) AS version_pair SELECT version_pair[1], toUInt32(splitByChar('.', version_pair[2])[1]) > 3 \ No newline at end of file +WITH splitByChar(' ', getOSKernelVersion()) AS version_pair SELECT version_pair[1] \ No newline at end of file From f908531b76fac8f42fe5eeeb4496e7535142b633 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 9 Oct 2021 19:12:47 +0300 Subject: [PATCH 346/950] Fix debug build. --- src/DataStreams/NativeReader.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/DataStreams/NativeReader.cpp b/src/DataStreams/NativeReader.cpp index 92c37a3f764..079dff80eae 100644 --- a/src/DataStreams/NativeReader.cpp +++ b/src/DataStreams/NativeReader.cpp @@ -61,14 +61,6 @@ void NativeReader::resetParser() { istr_concrete = nullptr; use_index = false; - -#ifndef NDEBUG - read_prefix_is_called = false; - read_suffix_is_called = false; -#endif - - // is_cancelled.store(false); - // is_killed.store(false); } void NativeReader::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) From 0828ecae975d8f92756e28430d610c9bdce7b8d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Oct 2021 19:46:36 +0300 Subject: [PATCH 347/950] Better exception message while reading column from Arrow-supported formats --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 13 ++++++++++++- .../Formats/Impl/ParquetBlockInputFormat.cpp | 1 + 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index feb826d5aa0..3dada0d62be 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -568,7 +569,17 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrgetName(), header_column.type->getName())); + throw; + } + column.type = header_column.type; num_rows = column.column->size(); columns_list.push_back(std::move(column.column)); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 5137318c6e1..8f088a3f84a 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -15,6 +15,7 @@ #include + namespace DB { From c61f09c4dc1df370fd73b00c450a029afc3a6544 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 9 Oct 2021 18:20:13 +0300 Subject: [PATCH 348/950] fix test for replication consistency --- tests/queries/0_stateless/replication.lib | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index e7fa2dd5404..61491630f46 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -40,6 +40,17 @@ function check_replication_consistency() table_name_prefix=$1 check_query_part=$2 + # Wait for all queries to finish (query may still be running if thread is killed by timeout) + num_tries=0 + while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%'") -ne 1 ]]; do + sleep 0.5; + num_tries=$((num_tries-1)) + if [ $num_tries -eq 100 ]; then + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE current_database=currentDatabase() AND query LIKE '%$table_name_prefix%' FORMAT Vertical" + break + fi + done + # Do not check anything if all replicas are readonly, # because is this case all replicas are probably lost (it may happen and it's not a bug) res=$($CLICKHOUSE_CLIENT -q "SELECT count() - sum(is_readonly) FROM system.replicas WHERE database=currentDatabase() AND table LIKE '$table_name_prefix%'") From 7f5852a7114e5e4da08f363ec81dcbfe4079d2f0 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 9 Oct 2021 18:37:28 +0000 Subject: [PATCH 349/950] New buildId variant Links from Distributed --- docs/en/engines/table-engines/special/distributed.md | 3 ++- docs/en/sql-reference/functions/other-functions.md | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 368849359ef..9503944a7a8 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -197,5 +197,6 @@ When the `max_parallel_replicas` option is enabled, query processing is parallel - [Virtual columns](../../../engines/table-engines/special/index.md#table_engines-virtual_columns) - [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) +- [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions + -[Original article](https://clickhouse.com/docs/en/operations/table_engines/distributed/) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 9828c91909b..6864ba7705b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -701,7 +701,7 @@ If it is executed in the context of a distributed table, then it generates a nor ## buildId() {#buildid} -Returns the compiler build id of the running binary. +Returns the build ID generated by a compiler for the running ClickHouse server binary. If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. From 2b272f5781ac22659dc8da6e4f657a359a74dac6 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 9 Oct 2021 19:17:02 +0000 Subject: [PATCH 350/950] Virtual column in Distributed updated, link fixed, links added Translated that part --- docs/en/engines/table-engines/special/distributed.md | 8 ++++---- docs/ru/engines/table-engines/special/distributed.md | 12 ++++++++++++ 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 9503944a7a8..6593a5dc17f 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -188,15 +188,15 @@ When the `max_parallel_replicas` option is enabled, query processing is parallel ## Virtual Columns {#virtual-columns} -- `_shard_num` — Contains the `shard_num` (from `system.clusters`). Type: [UInt32](../../../sql-reference/data-types/int-uint.md). +- `_shard_num` — Contains the `shard_num` value from the table `system.clusters`. Type: [UInt32](../../../sql-reference/data-types/int-uint.md). !!! note "Note" - Since [`remote`](../../../sql-reference/table-functions/remote.md)/`cluster` table functions internally create temporary instance of the same Distributed engine, `_shard_num` is available there too. + Since [remote](../../../sql-reference/table-functions/remote.md) and [cluster](../../../sql-reference/table-functions/cluster.md) table functions internally create temporary Distributed table, `_shard_num` is available there too. **See Also** -- [Virtual columns](../../../engines/table-engines/special/index.md#table_engines-virtual_columns) -- [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) +- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description +- [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting - [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions diff --git a/docs/ru/engines/table-engines/special/distributed.md b/docs/ru/engines/table-engines/special/distributed.md index b1f6f56623d..ff1dc7c4057 100644 --- a/docs/ru/engines/table-engines/special/distributed.md +++ b/docs/ru/engines/table-engines/special/distributed.md @@ -136,3 +136,15 @@ logs - имя кластера в конфигурационном файле с При выставлении опции max_parallel_replicas выполнение запроса распараллеливается по всем репликам внутри одного шарда. Подробнее смотрите раздел [max_parallel_replicas](../../../operations/settings/settings.md#settings-max_parallel_replicas). +## Виртуальные столбцы {#virtual-columns} + +- `_shard_num` — содержит значение `shard_num` из таблицы `system.clusters`. Тип: [UInt32](../../../sql-reference/data-types/int-uint.md). + +!!! note "Примечание" + Так как табличные функции [remote](../../../sql-reference/table-functions/remote.md) и [cluster](../../../sql-reference/table-functions/cluster.md) создают временную таблицу на движке `Distributed`, то в ней также доступен столбец `_shard_num`. + +**Смотрите также** + +- общее описание [виртуальных столбцов](../../../engines/table-engines/index.md#table_engines-virtual_columns) +- настройка [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) +- функции [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) и [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) From 17552931af3806c528c34a3788b53d572f6726e0 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 9 Oct 2021 19:27:24 +0000 Subject: [PATCH 351/950] ru other functuins addons --- docs/ru/sql-reference/functions/date-time-functions.md | 1 + docs/ru/sql-reference/functions/other-functions.md | 9 +++++++++ 2 files changed, 10 insertions(+) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 282962b9e3f..d4777faf354 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,6 +26,7 @@ SELECT ## timeZone {#timezone} Возвращает часовой пояс сервера. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index d3e0f8b946e..31e81b04330 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -8,6 +8,7 @@ toc_title: "Прочие функции" ## hostName() {#hostname} Возвращает строку - имя хоста, на котором эта функция была выполнена. При распределённой обработке запроса, это будет имя хоста удалённого сервера, если функция выполняется на удалённом сервере. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. ## getMacro {#getmacro} @@ -643,10 +644,17 @@ SELECT ## uptime() {#uptime} Возвращает аптайм сервера в секундах. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. ## version() {#version} Возвращает версию сервера в виде строки. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. + +## buildId() {#buildid} + +Возвращает ID сборки, сгенерированный компилятором для запущенного сервера ClickHouse. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. ## rowNumberInBlock {#function-rownumberinblock} @@ -2304,3 +2312,4 @@ SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0 │ 1 │ └─────────┘ ``` + From b9969232f54f4e947aff6aa07eea165bf00695da Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 10 Oct 2021 01:12:36 +0300 Subject: [PATCH 352/950] Update 02095_function_get_os_kernel_version.sql --- .../0_stateless/02095_function_get_os_kernel_version.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql b/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql index ed38abb64a9..d62b360f7e0 100644 --- a/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql +++ b/tests/queries/0_stateless/02095_function_get_os_kernel_version.sql @@ -1 +1 @@ -WITH splitByChar(' ', getOSKernelVersion()) AS version_pair SELECT version_pair[1] \ No newline at end of file +WITH splitByChar(' ', getOSKernelVersion()) AS version_pair SELECT version_pair[1] From cc1fbe27a7e5d0ad35678a9c78a3312f580e8a77 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 03:40:51 +0300 Subject: [PATCH 353/950] Fix build due to conflicts in serverConstants Refs: #29755 Refs: #29913 --- src/Functions/serverConstants.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 8fc103b2382..900a6dbf2a9 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -106,7 +106,7 @@ namespace { public: static constexpr auto name = "getOSKernelVersion"; - explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionConstantBase(context, Poco::Environment::osName() + " " + Poco::Environment::osVersion()) {} + explicit FunctionGetOSKernelVersion(ContextPtr context) : FunctionConstantBase(Poco::Environment::osName() + " " + Poco::Environment::osVersion(), context->isDistributed()) {} static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; #endif From 4bc90d1dd7dbd4b8a9b6920d00ca24e8b160358e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 01:51:41 +0300 Subject: [PATCH 354/950] Fix data-race between flush() and startup() in StorageBuffer Stress tests found [1], TSan report: ================== WARNING: ThreadSanitizer: data race (pid=485) Read of size 8 at 0x7b5001280bd8 by thread T567 (mutexes: write M612061890855345680): 1 std::__1::shared_ptr::operator bool() const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2851:62 (clickhouse+0x159140a6) 2 bool std::__1::operator!=() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3447:30 (clickhouse+0x159140a6) 3 DB::BackgroundSchedulePoolTaskHolder::operator bool() const obj-x86_64-linux-gnu/../src/Core/BackgroundSchedulePool.h:164:46 (clickhouse+0x159140a6) 4 DB::StorageBuffer::flush() obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:675:10 (clickhouse+0x159140a6) Previous write of size 8 at 0x7b5001280bd8 by thread T586 (mutexes: write M191819750614415520): 2 std::__1::shared_ptr::operator=(std::__1::shared_ptr&&) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3243:34 (clickhouse+0x15913e22) 3 DB::BackgroundSchedulePoolTaskHolder::operator=() obj-x86_64-linux-gnu/../src/Core/BackgroundSchedulePool.h:156:110 (clickhouse+0x15913e22) 4 DB::StorageBuffer::startup() obj-x86_64-linux-gnu/../src/Storages/StorageBuffer.cpp:668:18 (clickhouse+0x15913e22) 5 DB::InterpreterCreateQuery::doCreateTable() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterCreateQuery.cpp:1092:10 (clickhouse+0x149bef7b) 6 DB::InterpreterCreateQuery::createTable() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterCreateQuery.cpp:952:20 (clickhouse+0x149ba9f5) 7 DB::InterpreterCreateQuery::execute() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterCreateQuery.cpp:1302:16 (clickhouse+0x149c1086) [1]: https://clickhouse-test-reports.s3.yandex.net/0/1c9778603ff49563d1d3d0d357de0608167e504d/stress_test_(thread).html Fixes: #29416 --- src/Storages/StorageBuffer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9747ea2dbf6..c3ce70955bf 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -130,6 +130,8 @@ StorageBuffer::StorageBuffer( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + + flush_handle = bg_pool.createTask(log->name() + "/Bg", [this]{ backgroundFlush(); }); } @@ -667,7 +669,6 @@ void StorageBuffer::startup() LOG_WARNING(log, "Storage {} is run with readonly settings, it will not be able to insert data. Set appropriate buffer_profile to fix this.", getName()); } - flush_handle = bg_pool.createTask(log->name() + "/Bg", [this]{ backgroundFlush(); }); flush_handle->activateAndSchedule(); } From 2bab572caf91d2fc8962f409fea34149ecec8c95 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 00:46:28 +0300 Subject: [PATCH 355/950] Fix lock-order-inversion between periodic dictionary reload and config reload Integration tests found [1], TSan report: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11) Cycle in lock order graph: M3152 (0x7b9000000058) => M3153 (0x7b9000000438) => M3152 Mutex M3153 acquired here while holding mutex M3152 in main thread: 3 std::__1::lock_guard::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse+0x15716b81) 4 DB::ExternalLoader::addConfigRepository() const obj-x86_64-linux-gnu/../src/Interpreters/ExternalLoader.cpp:1283:21 (clickhouse+0x15716b81) 5 DB::Context::loadOrReloadDictionaries() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1453:62 (clickhouse+0x155d7bad) 6 DB::Server::main()::$_1::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:852:29 (clickhouse+0x9b1944c) 13 DB::ConfigReloader::reloadIfNewer() obj-x86_64-linux-gnu/../src/Common/Config/ConfigReloader.cpp:137:13 (clickhouse+0x17045e2e) 14 DB::ConfigReloader::ConfigReloader() obj-x86_64-linux-gnu/../src/Common/Config/ConfigReloader.cpp:33:9 (clickhouse+0x17044e51) 16 DB::Server::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:803:33 (clickhouse+0x9b0c41d) 17 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse+0x19ffc08b) 18 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:405:25 (clickhouse+0x9b03ebe) 19 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse+0x1a01c246) 20 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:183:20 (clickhouse+0x9b02576) 21 main obj-x86_64-linux-gnu/../programs/main.cpp:372:12 (clickhouse+0x9b00a3a) Mutex M3152 acquired here while holding mutex M3153 in thread T2: 3 std::__1::lock_guard::lock_guard() obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse+0x155d63b8) 4 DB::Context::getExternalDictionariesLoader() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1337:21 (clickhouse+0x155d63b8) 5 DB::Context::getExternalDictionariesLoader() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1332:41 (clickhouse+0x155d6359) 6 DB::DatabaseDictionary::tryGetTable() const obj-x86_64-inux-gnu/../src/Databases/DatabaseDictionary.cpp:76:38 (clickhouse+0x157819ad) 7 DB::DatabaseCatalog::getTableImpl() const obj-x86_64-linux-gnu/../src/Interpreters/DatabaseCatalog.cpp:285:28 (clickhouse+0x1564a1fa) 8 DB::DatabaseCatalog::getTable() const obj-x86_64-linux-gnu/../src/Interpreters/DatabaseCatalog.cpp:656:16 (clickhouse+0x1564fa2a) 9 DB::JoinedTables::getLeftTableStorage() obj-x86_64-linux-gnu/../src/Interpreters/JoinedTables.cpp:219:40 (clickhouse+0x15eeef45) 10 DB::InterpreterSelectQuery::InterpreterSelectQuery() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:321:33 (clickhouse+0x15b792be) 19 DB::ClickHouseDictionarySource::doInvalidateQuery() const obj-x86_64-linux-gnu/../src/Dictionaries/ClickHouseDictionarySource.cpp:207:36 (clickhouse+0x12872d2d) 20 DB::ClickHouseDictionarySource::isModified() const obj-x86_64-linux-gnu/../src/Dictionaries/ClickHouseDictionarySource.cpp:144:25 (clickhouse+0x12872534) 21 DB::IDictionary::isModified() const (clickhouse+0x128ce39b) 22 DB::ExternalLoader::LoadingDispatcher::reloadOutdated() obj-x86_64-linux-gnu/../src/Interpreters/ExternalLoader.cpp:660:50 (clickhouse+0x157305f7) 23 DB::ExternalLoader::PeriodicUpdater::doPeriodicUpdates() obj-x86_64-linux-gnu/../src/Interpreters/ExternalLoader.cpp:1248:36 (clickhouse+0x1572fff7) [1]: https://clickhouse-test-reports.s3.yandex.net/29856/42ca2b4bb241827edf69bbd6938d6b19c31935f1/integration_tests_(thread).html#fail1 --- src/Interpreters/ExternalLoader.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index e4c8e46980d..dc8466f3c26 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -1243,8 +1243,10 @@ private: { lock.unlock(); { - std::lock_guard config_lock{config_mutex}; - loading_dispatcher.setConfiguration(config_files_reader.read()); + { + std::lock_guard config_lock{config_mutex}; + loading_dispatcher.setConfiguration(config_files_reader.read()); + } loading_dispatcher.reloadOutdated(); } lock.lock(); From daf4cc6aba1024c591e70e7f939c8802ce9b3885 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Oct 2021 04:09:17 +0300 Subject: [PATCH 356/950] Whitespaces --- cmake/arch.cmake | 1 - cmake/dbms_glob_sources.cmake | 15 ++++----------- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/cmake/arch.cmake b/cmake/arch.cmake index 60e0346dbbf..00cc16fbd10 100644 --- a/cmake/arch.cmake +++ b/cmake/arch.cmake @@ -13,7 +13,6 @@ endif () if ((ARCH_ARM AND NOT ARCH_AARCH64) OR ARCH_I386) message (FATAL_ERROR "32bit platforms are not supported") endif () - if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)") set (ARCH_PPC64LE 1) endif () diff --git a/cmake/dbms_glob_sources.cmake b/cmake/dbms_glob_sources.cmake index 9c8c53c63b6..0f5c6106b70 100644 --- a/cmake/dbms_glob_sources.cmake +++ b/cmake/dbms_glob_sources.cmake @@ -1,14 +1,7 @@ -if (CMAKE_VERSION VERSION_GREATER_EQUAL "3.12") - macro(add_glob cur_list) - file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} CONFIGURE_DEPENDS ${ARGN}) - list(APPEND ${cur_list} ${__tmp}) - endmacro() -else () - macro(add_glob cur_list) - file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN}) - list(APPEND ${cur_list} ${__tmp}) - endmacro() -endif () +macro(add_glob cur_list) + file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN}) + list(APPEND ${cur_list} ${__tmp}) +endmacro() macro(add_headers_and_sources prefix common_path) add_glob(${prefix}_headers ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h) From 4e6ed5c45c85166dcf7c98d4adaf02873b695183 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Oct 2021 04:10:52 +0300 Subject: [PATCH 357/950] Remove trash from SentryWriter --- base/daemon/SentryWriter.cpp | 40 ++---------------------------------- 1 file changed, 2 insertions(+), 38 deletions(-) diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index ad914ff8cf4..efd915b1e5b 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -64,41 +64,6 @@ void setExtras() sentry_set_extra("disk_free_space", sentry_value_new_string(formatReadableSizeWithBinarySuffix(fs::space(server_data_path).free).c_str())); } -void sentry_logger(sentry_level_e level, const char * message, va_list args, void *) -{ - auto * logger = &Poco::Logger::get("SentryWriter"); - size_t size = 1024; - char buffer[size]; -#ifdef __clang__ -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wformat-nonliteral" -#endif - if (vsnprintf(buffer, size, message, args) >= 0) - { -#ifdef __clang__ -#pragma clang diagnostic pop -#endif - switch (level) - { - case SENTRY_LEVEL_DEBUG: - logger->debug(buffer); - break; - case SENTRY_LEVEL_INFO: - logger->information(buffer); - break; - case SENTRY_LEVEL_WARNING: - logger->warning(buffer); - break; - case SENTRY_LEVEL_ERROR: - logger->error(buffer); - break; - case SENTRY_LEVEL_FATAL: - logger->fatal(buffer); - break; - } - } -} - } @@ -107,13 +72,13 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) bool enabled = false; bool debug = config.getBool("send_crash_reports.debug", false); auto * logger = &Poco::Logger::get("SentryWriter"); + if (config.getBool("send_crash_reports.enabled", false)) { if (debug || (strlen(VERSION_OFFICIAL) > 0)) //-V560 - { enabled = true; - } } + if (enabled) { server_data_path = config.getString("path", ""); @@ -126,7 +91,6 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) sentry_options_t * options = sentry_options_new(); /// will be freed by sentry_init or sentry_shutdown sentry_options_set_release(options, VERSION_STRING_SHORT); - sentry_options_set_logger(options, &sentry_logger, nullptr); if (debug) { sentry_options_set_debug(options, 1); From 2a20bf4909e38fae7acc1fd97646af10ba150696 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Oct 2021 04:16:32 +0300 Subject: [PATCH 358/950] Remove function bayesAB #26233 --- src/Functions/abtesting.cpp | 315 ------------------ src/Functions/abtesting.h | 35 -- src/Functions/registerFunctions.cpp | 7 - .../01411_bayesian_ab_testing.reference | 4 - .../0_stateless/01411_bayesian_ab_testing.sql | 6 - 5 files changed, 367 deletions(-) delete mode 100644 src/Functions/abtesting.cpp delete mode 100644 src/Functions/abtesting.h delete mode 100644 tests/queries/0_stateless/01411_bayesian_ab_testing.reference delete mode 100644 tests/queries/0_stateless/01411_bayesian_ab_testing.sql diff --git a/src/Functions/abtesting.cpp b/src/Functions/abtesting.cpp deleted file mode 100644 index 312fdf6fb48..00000000000 --- a/src/Functions/abtesting.cpp +++ /dev/null @@ -1,315 +0,0 @@ -#include - -#if !defined(ARCADIA_BUILD) && USE_STATS - -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#define STATS_ENABLE_STDVEC_WRAPPERS -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int BAD_ARGUMENTS; -} - -static const String BETA = "beta"; -static const String GAMMA = "gamma"; - -template -Variants bayesian_ab_test(String distribution, PODArray & xs, PODArray & ys) -{ - const size_t r = 1000, c = 100; - - Variants variants(xs.size(), {0.0, 0.0, 0.0, 0.0}); - std::vector> samples_matrix; - - for (size_t i = 0; i < xs.size(); ++i) - { - variants[i].x = xs[i]; - variants[i].y = ys[i]; - } - - if (distribution == BETA) - { - Float64 alpha, beta; - - for (size_t i = 0; i < xs.size(); ++i) - if (xs[i] < ys[i]) - throw Exception("Conversions cannot be larger than trials", ErrorCodes::BAD_ARGUMENTS); - - for (size_t i = 0; i < xs.size(); ++i) - { - alpha = 1.0 + ys[i]; - beta = 1.0 + xs[i] - ys[i]; - - samples_matrix.emplace_back(stats::rbeta>(r, c, alpha, beta)); - } - } - else if (distribution == GAMMA) - { - Float64 shape, scale; - - for (size_t i = 0; i < xs.size(); ++i) - { - shape = 1.0 + xs[i]; - scale = 250.0 / (1 + 250.0 * ys[i]); - - std::vector samples = stats::rgamma>(r, c, shape, scale); - for (auto & sample : samples) - sample = 1 / sample; - samples_matrix.emplace_back(std::move(samples)); - } - } - - PODArray means; - for (auto & samples : samples_matrix) - { - Float64 total = 0.0; - for (auto sample : samples) - total += sample; - means.push_back(total / samples.size()); - } - - // Beats control - for (size_t i = 1; i < xs.size(); ++i) - { - for (size_t n = 0; n < r * c; ++n) - { - if (higher_is_better) - { - if (samples_matrix[i][n] > samples_matrix[0][n]) - ++variants[i].beats_control; - } - else - { - if (samples_matrix[i][n] < samples_matrix[0][n]) - ++variants[i].beats_control; - } - } - } - - for (auto & variant : variants) - variant.beats_control = static_cast(variant.beats_control) / r / c; - - // To be best - PODArray count_m(xs.size(), 0); - PODArray row(xs.size(), 0); - - for (size_t n = 0; n < r * c; ++n) - { - for (size_t i = 0; i < xs.size(); ++i) - row[i] = samples_matrix[i][n]; - - Float64 m; - if (higher_is_better) - m = *std::max_element(row.begin(), row.end()); - else - m = *std::min_element(row.begin(), row.end()); - - for (size_t i = 0; i < xs.size(); ++i) - { - if (m == samples_matrix[i][n]) - { - ++variants[i].best; - break; - } - } - } - - for (auto & variant : variants) - variant.best = static_cast(variant.best) / r / c; - - return variants; -} - -String convertToJson(const PODArray & variant_names, const Variants & variants) -{ - FormatSettings settings; - - WriteBufferFromOwnString buf; - - writeCString("{\"data\":[", buf); - for (size_t i = 0; i < variants.size(); ++i) - { - writeCString("{\"variant_name\":", buf); - writeJSONString(variant_names[i], buf, settings); - writeCString(",\"x\":", buf); - writeText(variants[i].x, buf); - writeCString(",\"y\":", buf); - writeText(variants[i].y, buf); - writeCString(",\"beats_control\":", buf); - writeText(variants[i].beats_control, buf); - writeCString(",\"to_be_best\":", buf); - writeText(variants[i].best, buf); - writeCString("}", buf); - if (i != variant_names.size() -1) - writeCString(",", buf); - } - writeCString("]}", buf); - - return buf.str(); -} - -class FunctionBayesAB : public IFunction -{ -public: - static constexpr auto name = "bayesAB"; - - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } - - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - size_t getNumberOfArguments() const override { return 5; } - - DataTypePtr getReturnTypeImpl(const DataTypes &) const override - { - return std::make_shared(); - } - - static bool toFloat64(const ColumnConst * col_const_arr, PODArray & output) - { - Array src_arr = col_const_arr->getValue(); - - for (size_t i = 0, size = src_arr.size(); i < size; ++i) - { - switch (src_arr[i].getType()) - { - case Field::Types::Int64: - output.push_back(static_cast(src_arr[i].get())); - break; - case Field::Types::UInt64: - output.push_back(static_cast(src_arr[i].get())); - break; - case Field::Types::Float64: - output.push_back(src_arr[i].get()); - break; - default: - return false; - } - } - - return true; - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - if (input_rows_count == 0) - return ColumnString::create(); - - PODArray xs, ys; - PODArray variant_names; - String dist; - bool higher_is_better; - - if (const ColumnConst * col_dist = checkAndGetColumnConst(arguments[0].column.get())) - { - dist = col_dist->getDataAt(0).data; - dist = Poco::toLower(dist); - if (dist != BETA && dist != GAMMA) - throw Exception("First argument for function " + getName() + " cannot be " + dist, ErrorCodes::BAD_ARGUMENTS); - } - else - throw Exception("First argument for function " + getName() + " must be Constant string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (const ColumnConst * col_higher_is_better = checkAndGetColumnConst(arguments[1].column.get())) - higher_is_better = col_higher_is_better->getBool(0); - else - throw Exception("Second argument for function " + getName() + " must be Constant boolean", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[2].column.get())) - { - Array src_arr = col_const_arr->getValue(); - - for (size_t i = 0; i < src_arr.size(); ++i) - { - if (src_arr[i].getType() != Field::Types::String) - throw Exception("Third argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - variant_names.push_back(src_arr[i].get()); - } - } - else - throw Exception("Third argument for function " + getName() + " must be Array of constant strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[3].column.get())) - { - if (!toFloat64(col_const_arr, xs)) - throw Exception("Forth and fifth Argument for function " + getName() + " must be Array of constant Numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - else - throw Exception("Forth argument for function " + getName() + " must be Array of constant numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[4].column.get())) - { - if (!toFloat64(col_const_arr, ys)) - throw Exception("Fifth Argument for function " + getName() + " must be Array of constant Numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - else - throw Exception("Fifth argument for function " + getName() + " must be Array of constant numbers", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (variant_names.size() != xs.size() || xs.size() != ys.size()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of arguments doesn't match: variant_names: {}, xs: {}, ys: {}", variant_names.size(), xs.size(), ys.size()); - - if (variant_names.size() < 2) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Sizes of arguments must be larger than 1. variant_names: {}, xs: {}, ys: {}", variant_names.size(), xs.size(), ys.size()); - - if (std::count_if(xs.begin(), xs.end(), [](Float64 v) { return v < 0; }) > 0 || - std::count_if(ys.begin(), ys.end(), [](Float64 v) { return v < 0; }) > 0) - throw Exception("Negative values don't allowed", ErrorCodes::BAD_ARGUMENTS); - - Variants variants; - if (higher_is_better) - variants = bayesian_ab_test(dist, xs, ys); - else - variants = bayesian_ab_test(dist, xs, ys); - - auto dst = ColumnString::create(); - std::string result_str = convertToJson(variant_names, variants); - dst->insertData(result_str.c_str(), result_str.length()); - return dst; - } -}; - -void registerFunctionBayesAB(FunctionFactory & factory) -{ - factory.registerFunction(); -} - -} - -#else - -namespace DB -{ - -class FunctionFactory; - -void registerFunctionBayesAB(FunctionFactory & /* factory */) -{ -} - -} - -#endif diff --git a/src/Functions/abtesting.h b/src/Functions/abtesting.h deleted file mode 100644 index b1f12e79437..00000000000 --- a/src/Functions/abtesting.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include - -#if !defined(ARCADIA_BUILD) && USE_STATS - -# include -# include - -# include -# include -# include - - -namespace DB -{ - -struct Variant -{ - Float64 x; - Float64 y; - Float64 beats_control; - Float64 best; -}; - -using Variants = PODArray; - -template -Variants bayesian_ab_test(String distribution, PODArray & xs, PODArray & ys); - -String convertToJson(const PODArray & variant_names, const Variants & variants); - -} - -#endif diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index 35193e9be8d..b2f038240aa 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -54,9 +54,6 @@ void registerFunctionTupleHammingDistance(FunctionFactory & factory); void registerFunctionsStringHash(FunctionFactory & factory); void registerFunctionValidateNestedArraySizes(FunctionFactory & factory); void registerFunctionsSnowflake(FunctionFactory & factory); -#if !defined(ARCADIA_BUILD) -void registerFunctionBayesAB(FunctionFactory &); -#endif void registerFunctionTid(FunctionFactory & factory); void registerFunctionLogTrace(FunctionFactory & factory); @@ -122,10 +119,6 @@ void registerFunctions() registerFunctionValidateNestedArraySizes(factory); registerFunctionsSnowflake(factory); -#if !defined(ARCADIA_BUILD) - registerFunctionBayesAB(factory); -#endif - #if USE_SSL registerFunctionEncrypt(factory); registerFunctionDecrypt(factory); diff --git a/tests/queries/0_stateless/01411_bayesian_ab_testing.reference b/tests/queries/0_stateless/01411_bayesian_ab_testing.reference deleted file mode 100644 index 98fb6a68656..00000000000 --- a/tests/queries/0_stateless/01411_bayesian_ab_testing.reference +++ /dev/null @@ -1,4 +0,0 @@ -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/01411_bayesian_ab_testing.sql b/tests/queries/0_stateless/01411_bayesian_ab_testing.sql deleted file mode 100644 index a4b03d76c51..00000000000 --- a/tests/queries/0_stateless/01411_bayesian_ab_testing.sql +++ /dev/null @@ -1,6 +0,0 @@ --- Tags: no-fasttest - -SELECT count() FROM (SELECT bayesAB('beta', 1, ['Control', 'A', 'B'], [3000.0, 3000.0, 2000.0], [1000.0, 1100.0, 800.0])); -SELECT count() FROM (SELECT bayesAB('gamma', 1, ['Control', 'A', 'B'], [3000.0, 3000.0, 2000.0], [1000.0, 1100.0, 800.0])); -SELECT count() FROM (SELECT bayesAB('beta', 0, ['Control', 'A', 'B'], [3000.0, 3000.0, 2000.0], [1000.0, 1100.0, 800.0])); -SELECT count() FROM (SELECT bayesAB('gamma', 0, ['Control', 'A', 'B'], [3000.0, 3000.0, 2000.0], [1000.0, 1100.0, 800.0])); From 103b3c91ba9d8026e4eb574dd5a11f9b075f0b26 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Oct 2021 04:21:07 +0300 Subject: [PATCH 359/950] Remove 'printf' function usage. --- src/Client/QueryFuzzer.cpp | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index aa9f89e47b5..ec267e44a99 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -144,7 +144,7 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - fprintf(stderr, "erased\n"); + std::cerr << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -153,12 +153,12 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - fprintf(stderr, "inserted (pos %zd)\n", pos); + std::cerr << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - fprintf(stderr, "inserted (0)\n"); + std::cerr << "inserted (0)\n"; } } @@ -278,7 +278,7 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - fprintf(stderr, "no random col!\n"); + std::cerr << "No random column.\n"; } } @@ -312,13 +312,9 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) : impl->children.begin() + fuzz_rand() % impl->children.size(); auto col = getRandomColumnLike(); if (col) - { impl->children.insert(pos, col); - } else - { - fprintf(stderr, "no random col!\n"); - } + std::cerr << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by From 1229a8b5284f409e1af6dfa6115da1c1f5da7647 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 10 Oct 2021 04:12:24 +0000 Subject: [PATCH 360/950] fix --- src/Parsers/ASTAlterQuery.cpp | 1 + .../0_stateless/01318_alter_add_constraint_format.reference | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index ab64c319807..53d53bf5ae1 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -521,6 +521,7 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState FormatStateStacked frame_nested = frame; frame_nested.need_parens = false; + frame_nested.expression_list_always_start_on_new_line = true; static_cast(command_list)->formatImplMultiline(settings, state, frame_nested); } diff --git a/tests/queries/0_stateless/01318_alter_add_constraint_format.reference b/tests/queries/0_stateless/01318_alter_add_constraint_format.reference index 9f58d161539..7a3b41536e0 100644 --- a/tests/queries/0_stateless/01318_alter_add_constraint_format.reference +++ b/tests/queries/0_stateless/01318_alter_add_constraint_format.reference @@ -1 +1,2 @@ -ALTER TABLE replicated_constraints1 ADD CONSTRAINT IF NOT EXISTS b_constraint CHECK b > 10 +ALTER TABLE replicated_constraints1 + ADD CONSTRAINT IF NOT EXISTS b_constraint CHECK b > 10 From 452b3b443d580112dc2acb49349e2d3acdbd30ff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 9 Oct 2021 15:31:13 +0300 Subject: [PATCH 361/950] copier: add ability to configure retries and delays between them The following options had been introduced: - max-table-tries - max-shard-partition-tries - max-shard-partition-piece-tries-for-alter - retry-delay-ms --- programs/copier/ClusterCopier.cpp | 22 +++++++++++----------- programs/copier/ClusterCopier.h | 26 +++++++++++++++++++++----- programs/copier/ClusterCopierApp.cpp | 18 +++++++++++++++++- programs/copier/ClusterCopierApp.h | 5 +++++ 4 files changed, 54 insertions(+), 17 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 30b99b69351..89ef96e299e 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -87,7 +87,7 @@ decltype(auto) ClusterCopier::retry(T && func, UInt64 max_tries) if (try_number < max_tries) { tryLogCurrentException(log, "Will retry"); - std::this_thread::sleep_for(default_sleep_time); + std::this_thread::sleep_for(retry_delay_ms); } } } @@ -310,7 +310,7 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts) /// Retry table processing bool table_is_done = false; - for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) + for (UInt64 num_table_tries = 1; num_table_tries <= max_table_tries; ++num_table_tries) { if (tryProcessTable(timeouts, task_table)) { @@ -341,7 +341,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee const String & description, bool unprioritized) { - std::chrono::milliseconds current_sleep_time = default_sleep_time; + std::chrono::milliseconds current_sleep_time = retry_delay_ms; static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec if (unprioritized) @@ -367,7 +367,7 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee LOG_INFO(log, "Too many workers ({}, maximum {}). Postpone processing {}", stat.numChildren, task_cluster->max_workers, description); if (unprioritized) - current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); + current_sleep_time = std::min(max_sleep_time, current_sleep_time + retry_delay_ms); std::this_thread::sleep_for(current_sleep_time); num_bad_version_errors = 0; @@ -786,7 +786,7 @@ bool ClusterCopier::tryDropPartitionPiece( if (e.code == Coordination::Error::ZNODEEXISTS) { LOG_INFO(log, "Partition {} piece {} is cleaning now by somebody, sleep", task_partition.name, toString(current_piece_number)); - std::this_thread::sleep_for(default_sleep_time); + std::this_thread::sleep_for(retry_delay_ms); return false; } @@ -799,7 +799,7 @@ bool ClusterCopier::tryDropPartitionPiece( if (stat.numChildren != 0) { LOG_INFO(log, "Partition {} contains {} active workers while trying to drop it. Going to sleep.", task_partition.name, stat.numChildren); - std::this_thread::sleep_for(default_sleep_time); + std::this_thread::sleep_for(retry_delay_ms); return false; } else @@ -1006,7 +1006,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab task_status = TaskStatus::Error; bool was_error = false; has_shard_to_process = true; - for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + for (UInt64 try_num = 1; try_num <= max_shard_partition_tries; ++try_num) { task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); @@ -1021,7 +1021,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab break; /// Repeat on errors - std::this_thread::sleep_for(default_sleep_time); + std::this_thread::sleep_for(retry_delay_ms); } if (task_status == TaskStatus::Error) @@ -1069,7 +1069,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab break; /// Repeat on errors. - std::this_thread::sleep_for(default_sleep_time); + std::this_thread::sleep_for(retry_delay_ms); } catch (...) { @@ -1110,7 +1110,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab if (!table_is_done) { - LOG_INFO(log, "Table {} is not processed yet.Copied {} of {}, will retry", task_table.table_id, finished_partitions, required_partitions); + LOG_INFO(log, "Table {} is not processed yet. Copied {} of {}, will retry", task_table.table_id, finished_partitions, required_partitions); } else { @@ -1213,7 +1213,7 @@ TaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTim break; /// Repeat on errors - std::this_thread::sleep_for(default_sleep_time); + std::this_thread::sleep_for(retry_delay_ms); } was_active_pieces = (res == TaskStatus::Active); diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 387b089724a..b354fc59eee 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -65,6 +65,23 @@ public: experimental_use_sample_offset = value; } + void setMaxTableTries(UInt64 tries) + { + max_table_tries = tries; + } + void setMaxShardPartitionTries(UInt64 tries) + { + max_shard_partition_tries = tries; + } + void setMaxShardPartitionPieceTriesForAlter(UInt64 tries) + { + max_shard_partition_piece_tries_for_alter = tries; + } + void setRetryDelayMs(std::chrono::milliseconds ms) + { + retry_delay_ms = ms; + } + protected: String getWorkersPath() const @@ -123,10 +140,6 @@ protected: bool tryDropPartitionPiece(ShardPartition & task_partition, size_t current_piece_number, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); - static constexpr UInt64 max_table_tries = 3; - static constexpr UInt64 max_shard_partition_tries = 3; - static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10; - bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); TaskStatus tryCreateDestinationTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); @@ -218,6 +231,9 @@ private: Poco::Logger * log; - std::chrono::milliseconds default_sleep_time{1000}; + UInt64 max_table_tries = 3; + UInt64 max_shard_partition_tries = 3; + UInt64 max_shard_partition_piece_tries_for_alter = 10; + std::chrono::milliseconds retry_delay_ms{1000}; }; } diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index b8714d0851d..8d7e4abce51 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -31,6 +31,10 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0); base_dir = (config().has("base-dir")) ? config().getString("base-dir") : fs::current_path().string(); + max_table_tries = std::max(config().getUInt("max-table-tries", 3), 1); + max_shard_partition_tries = std::max(config().getUInt("max-shard-partition-tries", 3), 1); + max_shard_partition_piece_tries_for_alter = std::max(config().getUInt("max-shard-partition-piece-tries-for-alter", 10), 1); + retry_delay_ms = std::chrono::milliseconds(std::max(config().getUInt("retry-delay-ms", 1000), 100)); if (config().has("experimental-use-sample-offset")) experimental_use_sample_offset = config().getBool("experimental-use-sample-offset"); @@ -100,6 +104,15 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) .argument("experimental-use-sample-offset").binding("experimental-use-sample-offset")); options.addOption(Poco::Util::Option("status", "", "Get for status for current execution").binding("status")); + options.addOption(Poco::Util::Option("max-table-tries", "", "Number of tries for the copy table task") + .argument("max-table-tries").binding("max-table-tries")); + options.addOption(Poco::Util::Option("max-shard-partition-tries", "", "Number of tries for the copy one partition task") + .argument("max-shard-partition-tries").binding("max-shard-partition-tries")); + options.addOption(Poco::Util::Option("max-shard-partition-piece-tries-for-alter", "", "Number of tries for final ALTER ATTACH to destination table") + .argument("max-shard-partition-piece-tries-for-alter").binding("max-shard-partition-piece-tries-for-alter")); + options.addOption(Poco::Util::Option("retry-delay-ms", "", "Delay between task retries") + .argument("retry-delay-ms").binding("retry-delay-ms")); + using Me = std::decay_t; options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help") .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); @@ -161,7 +174,10 @@ void ClusterCopierApp::mainImpl() copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); copier->setMoveFaultProbability(move_fault_probability); - + copier->setMaxTableTries(max_table_tries); + copier->setMaxShardPartitionTries(max_shard_partition_tries); + copier->setMaxShardPartitionPieceTriesForAlter(max_shard_partition_piece_tries_for_alter); + copier->setRetryDelayMs(retry_delay_ms); copier->setExperimentalUseSampleOffset(experimental_use_sample_offset); auto task_file = config().getString("task-file", ""); diff --git a/programs/copier/ClusterCopierApp.h b/programs/copier/ClusterCopierApp.h index cce07e338c0..d447cd96149 100644 --- a/programs/copier/ClusterCopierApp.h +++ b/programs/copier/ClusterCopierApp.h @@ -83,6 +83,11 @@ private: double move_fault_probability = 0.0; bool is_help = false; + UInt64 max_table_tries = 3; + UInt64 max_shard_partition_tries = 3; + UInt64 max_shard_partition_piece_tries_for_alter = 10; + std::chrono::milliseconds retry_delay_ms{1000}; + bool experimental_use_sample_offset{false}; std::string base_dir; From 38de1833793b794f1528f2daff0a0f05363f3303 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 09:33:40 +0300 Subject: [PATCH 362/950] Make 00984_parser_stack_overflow less flaky According to query_log/trace_log the problem is real: 2021.10.10 07:52:37.702364 [ 3736 ] {c6a219f2-3df8-4474-8324-bb307ee7a7a9} executeQuery: (from [::1]:38096) (comment: /usr/share/clickhouse-test/queries/0_stateless/00984_parser_stack_overflow.sh) SELECT [[[... 2021.10.10 07:53:11.407949 [ 3736 ] {c6a219f2-3df8-4474-8324-bb307ee7a7a9} executeQuery: Code: 306. DB::Exception: Maximum parse depth (1000) exceeded. Consider rising max_parser_depth ... From trace_log: 2021-10-10 07:52:37.958938 DB::Dwarf::findDebugInfoOffset 2021-10-10 07:53:12.583248 DB::Dwarf::findDebugInfoOffset CI: https://clickhouse-test-reports.s3.yandex.net/29928/2bab572caf91d2fc8962f409fea34149ecec8c95/functional_stateless_tests_(debug).html#fail1 --- tests/queries/0_stateless/00984_parser_stack_overflow.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00984_parser_stack_overflow.sh b/tests/queries/0_stateless/00984_parser_stack_overflow.sh index 167678db5ec..329e51e774a 100755 --- a/tests/queries/0_stateless/00984_parser_stack_overflow.sh +++ b/tests/queries/0_stateless/00984_parser_stack_overflow.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -CLICKHOUSE_CURL_TIMEOUT=30 +# Such a huge timeout mostly for debug build. +CLICKHOUSE_CURL_TIMEOUT=60 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 0c5d26b647e42b2acedff5f54501a360a05864d2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 09:53:44 +0300 Subject: [PATCH 363/950] Make 01085_max_distributed_connections less flaky --- .../0_stateless/01085_max_distributed_connections.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01085_max_distributed_connections.sh b/tests/queries/0_stateless/01085_max_distributed_connections.sh index d8efa792e9d..4ffcd980956 100755 --- a/tests/queries/0_stateless/01085_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01085_max_distributed_connections.sh @@ -10,13 +10,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) while true do opts=( - --max_distributed_connections 9 + --max_distributed_connections 20 --max_threads 1 - --query "SELECT sleepEachRow(1) FROM remote('127.{2..10}', system.one)" + --query "SELECT sleepEachRow(1) FROM remote('127.{2..21}', system.one)" --format Null ) - # 5 less then 9 seconds (9 streams), but long enough to cover possible load peaks + # 10 less then 20 seconds (20 streams), but long enough to cover possible load peaks # "$@" left to pass manual options (like --experimental_use_processors 0) during manual testing - timeout 5s ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" && break + timeout 10s ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" && break done From 1aeb5d55dd3b4eea4a4aff129a9c35bf1379b101 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Sun, 10 Oct 2021 12:55:37 +0300 Subject: [PATCH 364/950] Grep server log even if it contains binary data (#29903) * grep server log even if it contains binary data * Update cluster.py Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com> --- tests/integration/helpers/cluster.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5ba67085d73..51b7bfcbcb8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2073,11 +2073,11 @@ class ClickHouseInstance: def contains_in_log(self, substring, from_host=False): if from_host: result = subprocess_check_call(["bash", "-c", - f'[ -f {self.logs_dir}/clickhouse-server.log ] && grep "{substring}" {self.logs_dir}/clickhouse-server.log || true' + f'[ -f {self.logs_dir}/clickhouse-server.log ] && grep -a "{substring}" {self.logs_dir}/clickhouse-server.log || true' ]) else: result = self.exec_in_container(["bash", "-c", - f'[ -f /var/log/clickhouse-server/clickhouse-server.log ] && grep "{substring}" /var/log/clickhouse-server/clickhouse-server.log || true' + f'[ -f /var/log/clickhouse-server/clickhouse-server.log ] && grep -a "{substring}" /var/log/clickhouse-server/clickhouse-server.log || true' ]) return len(result) > 0 @@ -2085,18 +2085,18 @@ class ClickHouseInstance: logging.debug(f"grep in log called %s", substring) if from_host: result = subprocess_check_call(["bash", "-c", - f'grep "{substring}" {self.logs_dir}/clickhouse-server.log || true' + f'grep -a "{substring}" {self.logs_dir}/clickhouse-server.log || true' ]) else: result = self.exec_in_container(["bash", "-c", - f'grep "{substring}" /var/log/clickhouse-server/clickhouse-server.log || true' + f'grep -a "{substring}" /var/log/clickhouse-server/clickhouse-server.log || true' ]) logging.debug("grep result %s", result) return result def count_in_log(self, substring): result = self.exec_in_container( - ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log | wc -l'.format(substring)]) + ["bash", "-c", 'grep -a "{}" /var/log/clickhouse-server/clickhouse-server.log | wc -l'.format(substring)]) return result def wait_for_log_line(self, regexp, filename='/var/log/clickhouse-server/clickhouse-server.log', timeout=30, repetitions=1, look_behind_lines=100): From 28459b282de0813c7e4da02d5b64ab8341cd7e2d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Oct 2021 13:25:24 +0300 Subject: [PATCH 365/950] Fix shutdown of AccessControlManager. Now there cannot be configuration reloading after AccessControlManager has been destroyed. --- programs/server/Server.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 4ed5b114082..bfa402a6c21 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1036,6 +1036,10 @@ if (ThreadFuzzer::instance().isEffective()) server.start(); SCOPE_EXIT({ + /// Stop reloading of the main config. This must be done before `global_context->shutdown()` because + /// otherwise the reloading may pass a changed config to some destroyed parts of ContextSharedPart. + main_config_reloader.reset(); + /** Ask to cancel background jobs all table engines, * and also query_log. * It is important to do early, not in destructor of Context, because @@ -1076,9 +1080,6 @@ if (ThreadFuzzer::instance().isEffective()) /// Wait server pool to avoid use-after-free of destroyed context in the handlers server_pool.joinAll(); - // Uses a raw pointer to global context for getting ZooKeeper. - main_config_reloader.reset(); - /** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available. * At this moment, no one could own shared part of Context. */ From 689c9cdbb9beac89847c722308ad4881d0108129 Mon Sep 17 00:00:00 2001 From: mikael Date: Thu, 22 Jul 2021 14:06:45 +0200 Subject: [PATCH 366/950] Add uc_mcontext for FreeBSD aarch64 --- src/Common/StackTrace.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index 1406cf03fe4..fc995e435ec 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -186,6 +186,8 @@ static void * getCallerAddress(const ucontext_t & context) #elif defined(__APPLE__) && defined(__aarch64__) return reinterpret_cast(context.uc_mcontext->__ss.__pc); +#elif defined(__FreeBSD__) && defined(__aarch64__) + return reinterpret_cast(context.uc_mcontext.mc_gpregs.gp_elr); #elif defined(__aarch64__) return reinterpret_cast(context.uc_mcontext.pc); #elif defined(__powerpc64__) From 714ac8efc415a413fb437872bb52d0f3280b00b4 Mon Sep 17 00:00:00 2001 From: mikael Date: Thu, 22 Jul 2021 14:07:54 +0200 Subject: [PATCH 367/950] FreeBSD aarch64 doesn't implement sbrk --- .../jemalloc/internal/jemalloc_internal_defs.h.in | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 44c59e1ce7d..3db0e14b268 100644 --- a/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -161,7 +161,7 @@ * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage * segment (DSS). */ -#define JEMALLOC_DSS +/* #undef JEMALLOC_DSS */ /* Support memory filling (junk/zero). */ #define JEMALLOC_FILL From 5bd2fd0ba38dc8f7828045b7035729c7072a2ea6 Mon Sep 17 00:00:00 2001 From: mikael Date: Thu, 22 Jul 2021 14:08:45 +0200 Subject: [PATCH 368/950] Add the missing bits for FreeBSD aarch64 --- cmake/find/ldap.cmake | 1 + .../freebsd_aarch64/include/lber_types.h | 63 + .../freebsd_aarch64/include/ldap_config.h | 74 ++ .../freebsd_aarch64/include/ldap_features.h | 61 + .../freebsd_aarch64/include/portable.h | 1169 +++++++++++++++++ contrib/rocksdb-cmake/CMakeLists.txt | 5 + 6 files changed, 1373 insertions(+) create mode 100644 contrib/openldap-cmake/freebsd_aarch64/include/lber_types.h create mode 100644 contrib/openldap-cmake/freebsd_aarch64/include/ldap_config.h create mode 100644 contrib/openldap-cmake/freebsd_aarch64/include/ldap_features.h create mode 100644 contrib/openldap-cmake/freebsd_aarch64/include/portable.h diff --git a/cmake/find/ldap.cmake b/cmake/find/ldap.cmake index d8baea89429..71222d26c66 100644 --- a/cmake/find/ldap.cmake +++ b/cmake/find/ldap.cmake @@ -64,6 +64,7 @@ if (NOT OPENLDAP_FOUND AND NOT MISSING_INTERNAL_LDAP_LIBRARY) ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "aarch64" ) OR ( "${_system_name}" STREQUAL "linux" AND "${_system_processor}" STREQUAL "ppc64le" ) OR ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "x86_64" ) OR + ( "${_system_name}" STREQUAL "freebsd" AND "${_system_processor}" STREQUAL "aarch64" ) OR ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "x86_64" ) OR ( "${_system_name}" STREQUAL "darwin" AND "${_system_processor}" STREQUAL "aarch64" ) ) diff --git a/contrib/openldap-cmake/freebsd_aarch64/include/lber_types.h b/contrib/openldap-cmake/freebsd_aarch64/include/lber_types.h new file mode 100644 index 00000000000..dbd59430527 --- /dev/null +++ b/contrib/openldap-cmake/freebsd_aarch64/include/lber_types.h @@ -0,0 +1,63 @@ +/* include/lber_types.h. Generated from lber_types.hin by configure. */ +/* $OpenLDAP$ */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +/* + * LBER types + */ + +#ifndef _LBER_TYPES_H +#define _LBER_TYPES_H + +#include + +LDAP_BEGIN_DECL + +/* LBER boolean, enum, integers (32 bits or larger) */ +#define LBER_INT_T int + +/* LBER tags (32 bits or larger) */ +#define LBER_TAG_T long + +/* LBER socket descriptor */ +#define LBER_SOCKET_T int + +/* LBER lengths (32 bits or larger) */ +#define LBER_LEN_T long + +/* ------------------------------------------------------------ */ + +/* booleans, enumerations, and integers */ +typedef LBER_INT_T ber_int_t; + +/* signed and unsigned versions */ +typedef signed LBER_INT_T ber_sint_t; +typedef unsigned LBER_INT_T ber_uint_t; + +/* tags */ +typedef unsigned LBER_TAG_T ber_tag_t; + +/* "socket" descriptors */ +typedef LBER_SOCKET_T ber_socket_t; + +/* lengths */ +typedef unsigned LBER_LEN_T ber_len_t; + +/* signed lengths */ +typedef signed LBER_LEN_T ber_slen_t; + +LDAP_END_DECL + +#endif /* _LBER_TYPES_H */ diff --git a/contrib/openldap-cmake/freebsd_aarch64/include/ldap_config.h b/contrib/openldap-cmake/freebsd_aarch64/include/ldap_config.h new file mode 100644 index 00000000000..89f7b40b884 --- /dev/null +++ b/contrib/openldap-cmake/freebsd_aarch64/include/ldap_config.h @@ -0,0 +1,74 @@ +/* include/ldap_config.h. Generated from ldap_config.hin by configure. */ +/* $OpenLDAP$ */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +/* + * This file works in conjunction with OpenLDAP configure system. + * If you do no like the values below, adjust your configure options. + */ + +#ifndef _LDAP_CONFIG_H +#define _LDAP_CONFIG_H + +/* directory separator */ +#ifndef LDAP_DIRSEP +#ifndef _WIN32 +#define LDAP_DIRSEP "/" +#else +#define LDAP_DIRSEP "\\" +#endif +#endif + +/* directory for temporary files */ +#if defined(_WIN32) +# define LDAP_TMPDIR "C:\\." /* we don't have much of a choice */ +#elif defined( _P_tmpdir ) +# define LDAP_TMPDIR _P_tmpdir +#elif defined( P_tmpdir ) +# define LDAP_TMPDIR P_tmpdir +#elif defined( _PATH_TMPDIR ) +# define LDAP_TMPDIR _PATH_TMPDIR +#else +# define LDAP_TMPDIR LDAP_DIRSEP "tmp" +#endif + +/* directories */ +#ifndef LDAP_BINDIR +#define LDAP_BINDIR "/tmp/ldap-prefix/bin" +#endif +#ifndef LDAP_SBINDIR +#define LDAP_SBINDIR "/tmp/ldap-prefix/sbin" +#endif +#ifndef LDAP_DATADIR +#define LDAP_DATADIR "/tmp/ldap-prefix/share/openldap" +#endif +#ifndef LDAP_SYSCONFDIR +#define LDAP_SYSCONFDIR "/tmp/ldap-prefix/etc/openldap" +#endif +#ifndef LDAP_LIBEXECDIR +#define LDAP_LIBEXECDIR "/tmp/ldap-prefix/libexec" +#endif +#ifndef LDAP_MODULEDIR +#define LDAP_MODULEDIR "/tmp/ldap-prefix/libexec/openldap" +#endif +#ifndef LDAP_RUNDIR +#define LDAP_RUNDIR "/tmp/ldap-prefix/var" +#endif +#ifndef LDAP_LOCALEDIR +#define LDAP_LOCALEDIR "" +#endif + + +#endif /* _LDAP_CONFIG_H */ diff --git a/contrib/openldap-cmake/freebsd_aarch64/include/ldap_features.h b/contrib/openldap-cmake/freebsd_aarch64/include/ldap_features.h new file mode 100644 index 00000000000..f0cc7c3626f --- /dev/null +++ b/contrib/openldap-cmake/freebsd_aarch64/include/ldap_features.h @@ -0,0 +1,61 @@ +/* include/ldap_features.h. Generated from ldap_features.hin by configure. */ +/* $OpenLDAP$ */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation. + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +/* + * LDAP Features + */ + +#ifndef _LDAP_FEATURES_H +#define _LDAP_FEATURES_H 1 + +/* OpenLDAP API version macros */ +#define LDAP_VENDOR_VERSION 20501 +#define LDAP_VENDOR_VERSION_MAJOR 2 +#define LDAP_VENDOR_VERSION_MINOR 5 +#define LDAP_VENDOR_VERSION_PATCH X + +/* +** WORK IN PROGRESS! +** +** OpenLDAP reentrancy/thread-safeness should be dynamically +** checked using ldap_get_option(). +** +** The -lldap implementation is not thread-safe. +** +** The -lldap_r implementation is: +** LDAP_API_FEATURE_THREAD_SAFE (basic thread safety) +** but also be: +** LDAP_API_FEATURE_SESSION_THREAD_SAFE +** LDAP_API_FEATURE_OPERATION_THREAD_SAFE +** +** The preprocessor flag LDAP_API_FEATURE_X_OPENLDAP_THREAD_SAFE +** can be used to determine if -lldap_r is available at compile +** time. You must define LDAP_THREAD_SAFE if and only if you +** link with -lldap_r. +** +** If you fail to define LDAP_THREAD_SAFE when linking with +** -lldap_r or define LDAP_THREAD_SAFE when linking with -lldap, +** provided header definitions and declarations may be incorrect. +** +*/ + +/* is -lldap_r available or not */ +#define LDAP_API_FEATURE_X_OPENLDAP_THREAD_SAFE 1 + +/* LDAP v2 Referrals */ +/* #undef LDAP_API_FEATURE_X_OPENLDAP_V2_REFERRALS */ + +#endif /* LDAP_FEATURES */ diff --git a/contrib/openldap-cmake/freebsd_aarch64/include/portable.h b/contrib/openldap-cmake/freebsd_aarch64/include/portable.h new file mode 100644 index 00000000000..10a15fe3ca1 --- /dev/null +++ b/contrib/openldap-cmake/freebsd_aarch64/include/portable.h @@ -0,0 +1,1169 @@ +/* include/portable.h. Generated from portable.hin by configure. */ +/* include/portable.hin. Generated from configure.in by autoheader. */ + + +/* begin of portable.h.pre */ +/* This work is part of OpenLDAP Software . + * + * Copyright 1998-2020 The OpenLDAP Foundation + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted only as authorized by the OpenLDAP + * Public License. + * + * A copy of this license is available in the file LICENSE in the + * top-level directory of the distribution or, alternatively, at + * . + */ + +#ifndef _LDAP_PORTABLE_H +#define _LDAP_PORTABLE_H + +/* define this if needed to get reentrant functions */ +#ifndef REENTRANT +#define REENTRANT 1 +#endif +#ifndef _REENTRANT +#define _REENTRANT 1 +#endif + +/* define this if needed to get threadsafe functions */ +#ifndef THREADSAFE +#define THREADSAFE 1 +#endif +#ifndef _THREADSAFE +#define _THREADSAFE 1 +#endif +#ifndef THREAD_SAFE +#define THREAD_SAFE 1 +#endif +#ifndef _THREAD_SAFE +#define _THREAD_SAFE 1 +#endif + +#ifndef _SGI_MP_SOURCE +#define _SGI_MP_SOURCE 1 +#endif + +/* end of portable.h.pre */ + + +/* Define if building universal (internal helper macro) */ +/* #undef AC_APPLE_UNIVERSAL_BUILD */ + +/* define to use both and */ +/* #undef BOTH_STRINGS_H */ + +/* define if cross compiling */ +/* #undef CROSS_COMPILING */ + +/* set to the number of arguments ctime_r() expects */ +#define CTIME_R_NARGS 2 + +/* define if toupper() requires islower() */ +/* #undef C_UPPER_LOWER */ + +/* define if sys_errlist is not declared in stdio.h or errno.h */ +/* #undef DECL_SYS_ERRLIST */ + +/* define to enable slapi library */ +/* #undef ENABLE_SLAPI */ + +/* defined to be the EXE extension */ +#define EXEEXT "" + +/* set to the number of arguments gethostbyaddr_r() expects */ +#define GETHOSTBYADDR_R_NARGS 8 + +/* set to the number of arguments gethostbyname_r() expects */ +#define GETHOSTBYNAME_R_NARGS 6 + +/* Define to 1 if `TIOCGWINSZ' requires . */ +/* #undef GWINSZ_IN_SYS_IOCTL */ + +/* define if you have AIX security lib */ +/* #undef HAVE_AIX_SECURITY */ + +/* Define to 1 if you have the header file. */ +#define HAVE_ARPA_INET_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ARPA_NAMESER_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_ASSERT_H 1 + +/* Define to 1 if you have the `bcopy' function. */ +#define HAVE_BCOPY 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_BITS_TYPES_H */ + +/* Define to 1 if you have the `chroot' function. */ +#define HAVE_CHROOT 1 + +/* Define to 1 if you have the `closesocket' function. */ +/* #undef HAVE_CLOSESOCKET */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_CONIO_H */ + +/* define if crypt(3) is available */ +/* #undef HAVE_CRYPT */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_CRYPT_H */ + +/* define if crypt_r() is also available */ +/* #undef HAVE_CRYPT_R */ + +/* Define to 1 if you have the `ctime_r' function. */ +#define HAVE_CTIME_R 1 + +/* define if you have Cyrus SASL */ +/* #undef HAVE_CYRUS_SASL */ + +/* define if your system supports /dev/poll */ +/* #undef HAVE_DEVPOLL */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_DIRECT_H */ + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +#define HAVE_DIRENT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_DLFCN_H 1 + +/* Define to 1 if you don't have `vprintf' but do have `_doprnt.' */ +/* #undef HAVE_DOPRNT */ + +/* define if system uses EBCDIC instead of ASCII */ +/* #undef HAVE_EBCDIC */ + +/* Define to 1 if you have the `endgrent' function. */ +#define HAVE_ENDGRENT 1 + +/* Define to 1 if you have the `endpwent' function. */ +#define HAVE_ENDPWENT 1 + +/* define if your system supports epoll */ +/* #undef HAVE_EPOLL */ + +/* Define to 1 if you have the header file. */ +#define HAVE_ERRNO_H 1 + +/* Define to 1 if you have the `fcntl' function. */ +#define HAVE_FCNTL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_FCNTL_H 1 + +/* define if you actually have FreeBSD fetch(3) */ +/* #undef HAVE_FETCH */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_FILIO_H */ + +/* Define to 1 if you have the `flock' function. */ +#define HAVE_FLOCK 1 + +/* Define to 1 if you have the `fstat' function. */ +#define HAVE_FSTAT 1 + +/* Define to 1 if you have the `gai_strerror' function. */ +#define HAVE_GAI_STRERROR 1 + +/* Define to 1 if you have the `getaddrinfo' function. */ +#define HAVE_GETADDRINFO 1 + +/* Define to 1 if you have the `getdtablesize' function. */ +#define HAVE_GETDTABLESIZE 1 + +/* Define to 1 if you have the `geteuid' function. */ +#define HAVE_GETEUID 1 + +/* Define to 1 if you have the `getgrgid' function. */ +#define HAVE_GETGRGID 1 + +/* Define to 1 if you have the `gethostbyaddr_r' function. */ +#define HAVE_GETHOSTBYADDR_R 1 + +/* Define to 1 if you have the `gethostbyname_r' function. */ +#define HAVE_GETHOSTBYNAME_R 1 + +/* Define to 1 if you have the `gethostname' function. */ +#define HAVE_GETHOSTNAME 1 + +/* Define to 1 if you have the `getnameinfo' function. */ +#define HAVE_GETNAMEINFO 1 + +/* Define to 1 if you have the `getopt' function. */ +#define HAVE_GETOPT 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_GETOPT_H 1 + +/* Define to 1 if you have the `getpassphrase' function. */ +/* #undef HAVE_GETPASSPHRASE */ + +/* Define to 1 if you have the `getpeereid' function. */ +#define HAVE_GETPEEREID 1 + +/* Define to 1 if you have the `getpeerucred' function. */ +/* #undef HAVE_GETPEERUCRED */ + +/* Define to 1 if you have the `getpwnam' function. */ +#define HAVE_GETPWNAM 1 + +/* Define to 1 if you have the `getpwuid' function. */ +#define HAVE_GETPWUID 1 + +/* Define to 1 if you have the `getspnam' function. */ +/* #undef HAVE_GETSPNAM */ + +/* Define to 1 if you have the `gettimeofday' function. */ +#define HAVE_GETTIMEOFDAY 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GMP_H */ + +/* Define to 1 if you have the `gmtime_r' function. */ +#define HAVE_GMTIME_R 1 + +/* define if you have GNUtls */ +/* #undef HAVE_GNUTLS */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_GNUTLS_GNUTLS_H */ + +/* if you have GNU Pth */ +/* #undef HAVE_GNU_PTH */ + +/* Define to 1 if you have the header file. */ +#define HAVE_GRP_H 1 + +/* Define to 1 if you have the `hstrerror' function. */ +#define HAVE_HSTRERROR 1 + +/* define to you inet_aton(3) is available */ +#define HAVE_INET_ATON 1 + +/* Define to 1 if you have the `inet_ntoa_b' function. */ +/* #undef HAVE_INET_NTOA_B */ + +/* Define to 1 if you have the `inet_ntop' function. */ +#define HAVE_INET_NTOP 1 + +/* Define to 1 if you have the `initgroups' function. */ +#define HAVE_INITGROUPS 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_INTTYPES_H 1 + +/* Define to 1 if you have the `ioctl' function. */ +#define HAVE_IOCTL 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_IO_H */ + +/* define if your system supports kqueue */ +#define HAVE_KQUEUE 1 + +/* Define to 1 if you have the `gen' library (-lgen). */ +/* #undef HAVE_LIBGEN */ + +/* Define to 1 if you have the `gmp' library (-lgmp). */ +/* #undef HAVE_LIBGMP */ + +/* Define to 1 if you have the `inet' library (-linet). */ +/* #undef HAVE_LIBINET */ + +/* define if you have libtool -ltdl */ +/* #undef HAVE_LIBLTDL */ + +/* Define to 1 if you have the `net' library (-lnet). */ +/* #undef HAVE_LIBNET */ + +/* Define to 1 if you have the `nsl' library (-lnsl). */ +/* #undef HAVE_LIBNSL */ + +/* Define to 1 if you have the `nsl_s' library (-lnsl_s). */ +/* #undef HAVE_LIBNSL_S */ + +/* Define to 1 if you have the `socket' library (-lsocket). */ +/* #undef HAVE_LIBSOCKET */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LIBUTIL_H 1 + +/* Define to 1 if you have the `V3' library (-lV3). */ +/* #undef HAVE_LIBV3 */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LIMITS_H 1 + +/* if you have LinuxThreads */ +/* #undef HAVE_LINUX_THREADS */ + +/* Define to 1 if you have the header file. */ +#define HAVE_LOCALE_H 1 + +/* Define to 1 if you have the `localtime_r' function. */ +#define HAVE_LOCALTIME_R 1 + +/* Define to 1 if you have the `lockf' function. */ +#define HAVE_LOCKF 1 + +/* Define to 1 if the system has the type `long long'. */ +#define HAVE_LONG_LONG 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_LTDL_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_MALLOC_H */ + +/* Define to 1 if you have the `memcpy' function. */ +#define HAVE_MEMCPY 1 + +/* Define to 1 if you have the `memmove' function. */ +#define HAVE_MEMMOVE 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_MEMORY_H 1 + +/* Define to 1 if you have the `memrchr' function. */ +#define HAVE_MEMRCHR 1 + +/* Define to 1 if you have the `mkstemp' function. */ +#define HAVE_MKSTEMP 1 + +/* Define to 1 if you have the `mktemp' function. */ +#define HAVE_MKTEMP 1 + +/* define this if you have mkversion */ +#define HAVE_MKVERSION 1 + +/* Define to 1 if you have the header file, and it defines `DIR'. */ +/* #undef HAVE_NDIR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_NETINET_TCP_H 1 + +/* define if strerror_r returns char* instead of int */ +/* #undef HAVE_NONPOSIX_STRERROR_R */ + +/* if you have NT Event Log */ +/* #undef HAVE_NT_EVENT_LOG */ + +/* if you have NT Service Manager */ +/* #undef HAVE_NT_SERVICE_MANAGER */ + +/* if you have NT Threads */ +/* #undef HAVE_NT_THREADS */ + +/* define if you have OpenSSL */ +#define HAVE_OPENSSL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_OPENSSL_BN_H 1 + +/* define if you have OpenSSL with CRL checking capability */ +#define HAVE_OPENSSL_CRL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_OPENSSL_CRYPTO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_OPENSSL_SSL_H 1 + +/* Define to 1 if you have the `pipe' function. */ +#define HAVE_PIPE 1 + +/* Define to 1 if you have the `poll' function. */ +#define HAVE_POLL 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_POLL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PROCESS_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PSAP_H */ + +/* define to pthreads API spec revision */ +#define HAVE_PTHREADS 10 + +/* define if you have pthread_detach function */ +#define HAVE_PTHREAD_DETACH 1 + +/* Define to 1 if you have the `pthread_getconcurrency' function. */ +#define HAVE_PTHREAD_GETCONCURRENCY 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_PTHREAD_H 1 + +/* Define to 1 if you have the `pthread_kill' function. */ +#define HAVE_PTHREAD_KILL 1 + +/* Define to 1 if you have the `pthread_kill_other_threads_np' function. */ +/* #undef HAVE_PTHREAD_KILL_OTHER_THREADS_NP */ + +/* define if you have pthread_rwlock_destroy function */ +#define HAVE_PTHREAD_RWLOCK_DESTROY 1 + +/* Define to 1 if you have the `pthread_setconcurrency' function. */ +#define HAVE_PTHREAD_SETCONCURRENCY 1 + +/* Define to 1 if you have the `pthread_yield' function. */ +#define HAVE_PTHREAD_YIELD 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_PTH_H */ + +/* Define to 1 if the system has the type `ptrdiff_t'. */ +#define HAVE_PTRDIFF_T 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_PWD_H 1 + +/* Define to 1 if you have the `read' function. */ +#define HAVE_READ 1 + +/* Define to 1 if you have the `recv' function. */ +#define HAVE_RECV 1 + +/* Define to 1 if you have the `recvfrom' function. */ +#define HAVE_RECVFROM 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_REGEX_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_RESOLV_H */ + +/* define if you have res_query() */ +/* #undef HAVE_RES_QUERY */ + +/* define if OpenSSL needs RSAref */ +/* #undef HAVE_RSAREF */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SASL_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SASL_SASL_H */ + +/* define if your SASL library has sasl_version() */ +/* #undef HAVE_SASL_VERSION */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SCHED_H 1 + +/* Define to 1 if you have the `sched_yield' function. */ +#define HAVE_SCHED_YIELD 1 + +/* Define to 1 if you have the `send' function. */ +#define HAVE_SEND 1 + +/* Define to 1 if you have the `sendmsg' function. */ +#define HAVE_SENDMSG 1 + +/* Define to 1 if you have the `sendto' function. */ +#define HAVE_SENDTO 1 + +/* Define to 1 if you have the `setegid' function. */ +#define HAVE_SETEGID 1 + +/* Define to 1 if you have the `seteuid' function. */ +#define HAVE_SETEUID 1 + +/* Define to 1 if you have the `setgid' function. */ +#define HAVE_SETGID 1 + +/* Define to 1 if you have the `setpwfile' function. */ +/* #undef HAVE_SETPWFILE */ + +/* Define to 1 if you have the `setsid' function. */ +#define HAVE_SETSID 1 + +/* Define to 1 if you have the `setuid' function. */ +#define HAVE_SETUID 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SGTTY_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SHADOW_H */ + +/* Define to 1 if you have the `sigaction' function. */ +#define HAVE_SIGACTION 1 + +/* Define to 1 if you have the `signal' function. */ +#define HAVE_SIGNAL 1 + +/* Define to 1 if you have the `sigset' function. */ +#define HAVE_SIGSET 1 + +/* define if you have -lslp */ +/* #undef HAVE_SLP */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SLP_H */ + +/* Define to 1 if you have the `snprintf' function. */ +#define HAVE_SNPRINTF 1 + +/* if you have spawnlp() */ +/* #undef HAVE_SPAWNLP */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SQLEXT_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SQL_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_STDDEF_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDINT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STDLIB_H 1 + +/* Define to 1 if you have the `strdup' function. */ +#define HAVE_STRDUP 1 + +/* Define to 1 if you have the `strerror' function. */ +#define HAVE_STRERROR 1 + +/* Define to 1 if you have the `strerror_r' function. */ +#define HAVE_STRERROR_R 1 + +/* Define to 1 if you have the `strftime' function. */ +#define HAVE_STRFTIME 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRINGS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_STRING_H 1 + +/* Define to 1 if you have the `strpbrk' function. */ +#define HAVE_STRPBRK 1 + +/* Define to 1 if you have the `strrchr' function. */ +#define HAVE_STRRCHR 1 + +/* Define to 1 if you have the `strsep' function. */ +#define HAVE_STRSEP 1 + +/* Define to 1 if you have the `strspn' function. */ +#define HAVE_STRSPN 1 + +/* Define to 1 if you have the `strstr' function. */ +#define HAVE_STRSTR 1 + +/* Define to 1 if you have the `strtol' function. */ +#define HAVE_STRTOL 1 + +/* Define to 1 if you have the `strtoll' function. */ +#define HAVE_STRTOLL 1 + +/* Define to 1 if you have the `strtoq' function. */ +#define HAVE_STRTOQ 1 + +/* Define to 1 if you have the `strtoul' function. */ +#define HAVE_STRTOUL 1 + +/* Define to 1 if you have the `strtoull' function. */ +#define HAVE_STRTOULL 1 + +/* Define to 1 if you have the `strtouq' function. */ +#define HAVE_STRTOUQ 1 + +/* Define to 1 if `msg_accrightslen' is a member of `struct msghdr'. */ +/* #undef HAVE_STRUCT_MSGHDR_MSG_ACCRIGHTSLEN */ + +/* Define to 1 if `msg_control' is a member of `struct msghdr'. */ +/* #undef HAVE_STRUCT_MSGHDR_MSG_CONTROL */ + +/* Define to 1 if `pw_gecos' is a member of `struct passwd'. */ +#define HAVE_STRUCT_PASSWD_PW_GECOS 1 + +/* Define to 1 if `pw_passwd' is a member of `struct passwd'. */ +#define HAVE_STRUCT_PASSWD_PW_PASSWD 1 + +/* Define to 1 if `st_blksize' is a member of `struct stat'. */ +#define HAVE_STRUCT_STAT_ST_BLKSIZE 1 + +/* Define to 1 if `st_fstype' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_FSTYPE */ + +/* define to 1 if st_fstype is char * */ +/* #undef HAVE_STRUCT_STAT_ST_FSTYPE_CHAR */ + +/* define to 1 if st_fstype is int */ +/* #undef HAVE_STRUCT_STAT_ST_FSTYPE_INT */ + +/* Define to 1 if `st_vfstype' is a member of `struct stat'. */ +/* #undef HAVE_STRUCT_STAT_ST_VFSTYPE */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYNCH_H */ + +/* Define to 1 if you have the `sysconf' function. */ +#define HAVE_SYSCONF 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYSEXITS_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYSLOG_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_DEVPOLL_H */ + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +/* #undef HAVE_SYS_DIR_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_EPOLL_H */ + +/* define if you actually have sys_errlist in your libs */ +#define HAVE_SYS_ERRLIST 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_ERRNO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_EVENT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_FILE_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_FILIO_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_FSTYP_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_IOCTL_H 1 + +/* Define to 1 if you have the header file, and it defines `DIR'. + */ +/* #undef HAVE_SYS_NDIR_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_PARAM_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_POLL_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_PRIVGRP_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_RESOURCE_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SELECT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SOCKET_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_STAT_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_SYSLOG_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TIME_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_TYPES_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UCRED_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UIO_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_SYS_UN_H 1 + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_UUID_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_SYS_VMOUNT_H */ + +/* Define to 1 if you have that is POSIX.1 compatible. */ +#define HAVE_SYS_WAIT_H 1 + +/* define if you have -lwrap */ +/* #undef HAVE_TCPD */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_TCPD_H */ + +/* Define to 1 if you have the header file. */ +#define HAVE_TERMIOS_H 1 + +/* if you have Solaris LWP (thr) package */ +/* #undef HAVE_THR */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_THREAD_H */ + +/* Define to 1 if you have the `thr_getconcurrency' function. */ +/* #undef HAVE_THR_GETCONCURRENCY */ + +/* Define to 1 if you have the `thr_setconcurrency' function. */ +/* #undef HAVE_THR_SETCONCURRENCY */ + +/* Define to 1 if you have the `thr_yield' function. */ +/* #undef HAVE_THR_YIELD */ + +/* define if you have TLS */ +#define HAVE_TLS 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UNISTD_H 1 + +/* Define to 1 if you have the header file. */ +#define HAVE_UTIME_H 1 + +/* define if you have uuid_generate() */ +/* #undef HAVE_UUID_GENERATE */ + +/* define if you have uuid_to_str() */ +/* #undef HAVE_UUID_TO_STR */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_UUID_UUID_H */ + +/* Define to 1 if you have the `vprintf' function. */ +#define HAVE_VPRINTF 1 + +/* Define to 1 if you have the `vsnprintf' function. */ +#define HAVE_VSNPRINTF 1 + +/* Define to 1 if you have the `wait4' function. */ +#define HAVE_WAIT4 1 + +/* Define to 1 if you have the `waitpid' function. */ +#define HAVE_WAITPID 1 + +/* define if you have winsock */ +/* #undef HAVE_WINSOCK */ + +/* define if you have winsock2 */ +/* #undef HAVE_WINSOCK2 */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINSOCK2_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WINSOCK_H */ + +/* Define to 1 if you have the header file. */ +/* #undef HAVE_WIREDTIGER_H */ + +/* Define to 1 if you have the `write' function. */ +#define HAVE_WRITE 1 + +/* define if select implicitly yields */ +#define HAVE_YIELDING_SELECT 1 + +/* Define to 1 if you have the `_vsnprintf' function. */ +/* #undef HAVE__VSNPRINTF */ + +/* define to 32-bit or greater integer type */ +#define LBER_INT_T int + +/* define to large integer type */ +#define LBER_LEN_T long + +/* define to socket descriptor type */ +#define LBER_SOCKET_T int + +/* define to large integer type */ +#define LBER_TAG_T long + +/* define to 1 if library is thread safe */ +#define LDAP_API_FEATURE_X_OPENLDAP_THREAD_SAFE 1 + +/* define to LDAP VENDOR VERSION */ +/* #undef LDAP_API_FEATURE_X_OPENLDAP_V2_REFERRALS */ + +/* define this to add debugging code */ +/* #undef LDAP_DEBUG */ + +/* define if LDAP libs are dynamic */ +/* #undef LDAP_LIBS_DYNAMIC */ + +/* define to support PF_INET6 */ +#define LDAP_PF_INET6 1 + +/* define to support PF_LOCAL */ +#define LDAP_PF_LOCAL 1 + +/* define this to add SLAPI code */ +/* #undef LDAP_SLAPI */ + +/* define this to add syslog code */ +/* #undef LDAP_SYSLOG */ + +/* Version */ +#define LDAP_VENDOR_VERSION 20501 + +/* Major */ +#define LDAP_VENDOR_VERSION_MAJOR 2 + +/* Minor */ +#define LDAP_VENDOR_VERSION_MINOR 5 + +/* Patch */ +#define LDAP_VENDOR_VERSION_PATCH X + +/* Define to the sub-directory where libtool stores uninstalled libraries. */ +#define LT_OBJDIR ".libs/" + +/* define if memcmp is not 8-bit clean or is otherwise broken */ +/* #undef NEED_MEMCMP_REPLACEMENT */ + +/* define if you have (or want) no threads */ +/* #undef NO_THREADS */ + +/* define to use the original debug style */ +/* #undef OLD_DEBUG */ + +/* Package */ +#define OPENLDAP_PACKAGE "OpenLDAP" + +/* Version */ +#define OPENLDAP_VERSION "2.5.X" + +/* Define to the address where bug reports for this package should be sent. */ +#define PACKAGE_BUGREPORT "" + +/* Define to the full name of this package. */ +#define PACKAGE_NAME "" + +/* Define to the full name and version of this package. */ +#define PACKAGE_STRING "" + +/* Define to the one symbol short name of this package. */ +#define PACKAGE_TARNAME "" + +/* Define to the home page for this package. */ +#define PACKAGE_URL "" + +/* Define to the version of this package. */ +#define PACKAGE_VERSION "" + +/* define if sched_yield yields the entire process */ +/* #undef REPLACE_BROKEN_YIELD */ + +/* Define as the return type of signal handlers (`int' or `void'). */ +#define RETSIGTYPE void + +/* Define to the type of arg 1 for `select'. */ +#define SELECT_TYPE_ARG1 int + +/* Define to the type of args 2, 3 and 4 for `select'. */ +#define SELECT_TYPE_ARG234 (fd_set *) + +/* Define to the type of arg 5 for `select'. */ +#define SELECT_TYPE_ARG5 (struct timeval *) + +/* The size of `int', as computed by sizeof. */ +#define SIZEOF_INT 4 + +/* The size of `long', as computed by sizeof. */ +#define SIZEOF_LONG 8 + +/* The size of `long long', as computed by sizeof. */ +#define SIZEOF_LONG_LONG 8 + +/* The size of `short', as computed by sizeof. */ +#define SIZEOF_SHORT 2 + +/* The size of `wchar_t', as computed by sizeof. */ +#define SIZEOF_WCHAR_T 4 + +/* define to support per-object ACIs */ +/* #undef SLAPD_ACI_ENABLED */ + +/* define to support LDAP Async Metadirectory backend */ +/* #undef SLAPD_ASYNCMETA */ + +/* define to support cleartext passwords */ +/* #undef SLAPD_CLEARTEXT */ + +/* define to support crypt(3) passwords */ +/* #undef SLAPD_CRYPT */ + +/* define to support DNS SRV backend */ +/* #undef SLAPD_DNSSRV */ + +/* define to support LDAP backend */ +/* #undef SLAPD_LDAP */ + +/* define to support MDB backend */ +/* #undef SLAPD_MDB */ + +/* define to support LDAP Metadirectory backend */ +/* #undef SLAPD_META */ + +/* define to support modules */ +/* #undef SLAPD_MODULES */ + +/* dynamically linked module */ +#define SLAPD_MOD_DYNAMIC 2 + +/* statically linked module */ +#define SLAPD_MOD_STATIC 1 + +/* define to support cn=Monitor backend */ +/* #undef SLAPD_MONITOR */ + +/* define to support NDB backend */ +/* #undef SLAPD_NDB */ + +/* define to support NULL backend */ +/* #undef SLAPD_NULL */ + +/* define for In-Directory Access Logging overlay */ +/* #undef SLAPD_OVER_ACCESSLOG */ + +/* define for Audit Logging overlay */ +/* #undef SLAPD_OVER_AUDITLOG */ + +/* define for Automatic Certificate Authority overlay */ +/* #undef SLAPD_OVER_AUTOCA */ + +/* define for Collect overlay */ +/* #undef SLAPD_OVER_COLLECT */ + +/* define for Attribute Constraint overlay */ +/* #undef SLAPD_OVER_CONSTRAINT */ + +/* define for Dynamic Directory Services overlay */ +/* #undef SLAPD_OVER_DDS */ + +/* define for Dynamic Directory Services overlay */ +/* #undef SLAPD_OVER_DEREF */ + +/* define for Dynamic Group overlay */ +/* #undef SLAPD_OVER_DYNGROUP */ + +/* define for Dynamic List overlay */ +/* #undef SLAPD_OVER_DYNLIST */ + +/* define for Reverse Group Membership overlay */ +/* #undef SLAPD_OVER_MEMBEROF */ + +/* define for Password Policy overlay */ +/* #undef SLAPD_OVER_PPOLICY */ + +/* define for Proxy Cache overlay */ +/* #undef SLAPD_OVER_PROXYCACHE */ + +/* define for Referential Integrity overlay */ +/* #undef SLAPD_OVER_REFINT */ + +/* define for Return Code overlay */ +/* #undef SLAPD_OVER_RETCODE */ + +/* define for Rewrite/Remap overlay */ +/* #undef SLAPD_OVER_RWM */ + +/* define for Sequential Modify overlay */ +/* #undef SLAPD_OVER_SEQMOD */ + +/* define for ServerSideSort/VLV overlay */ +/* #undef SLAPD_OVER_SSSVLV */ + +/* define for Syncrepl Provider overlay */ +/* #undef SLAPD_OVER_SYNCPROV */ + +/* define for Translucent Proxy overlay */ +/* #undef SLAPD_OVER_TRANSLUCENT */ + +/* define for Attribute Uniqueness overlay */ +/* #undef SLAPD_OVER_UNIQUE */ + +/* define for Value Sorting overlay */ +/* #undef SLAPD_OVER_VALSORT */ + +/* define to support PASSWD backend */ +/* #undef SLAPD_PASSWD */ + +/* define to support PERL backend */ +/* #undef SLAPD_PERL */ + +/* define to support relay backend */ +/* #undef SLAPD_RELAY */ + +/* define to support reverse lookups */ +/* #undef SLAPD_RLOOKUPS */ + +/* define to support SHELL backend */ +/* #undef SLAPD_SHELL */ + +/* define to support SOCK backend */ +/* #undef SLAPD_SOCK */ + +/* define to support SASL passwords */ +/* #undef SLAPD_SPASSWD */ + +/* define to support SQL backend */ +/* #undef SLAPD_SQL */ + +/* define to support WiredTiger backend */ +/* #undef SLAPD_WT */ + +/* define to support run-time loadable ACL */ +/* #undef SLAP_DYNACL */ + +/* Define to 1 if you have the ANSI C header files. */ +#define STDC_HEADERS 1 + +/* Define to 1 if you can safely include both and . */ +#define TIME_WITH_SYS_TIME 1 + +/* Define to 1 if your declares `struct tm'. */ +/* #undef TM_IN_SYS_TIME */ + +/* set to urandom device */ +#define URANDOM_DEVICE "/dev/urandom" + +/* define to use OpenSSL BIGNUM for MP */ +/* #undef USE_MP_BIGNUM */ + +/* define to use GMP for MP */ +/* #undef USE_MP_GMP */ + +/* define to use 'long' for MP */ +/* #undef USE_MP_LONG */ + +/* define to use 'long long' for MP */ +/* #undef USE_MP_LONG_LONG */ + +/* Define WORDS_BIGENDIAN to 1 if your processor stores words with the most + significant byte first (like Motorola and SPARC, unlike Intel). */ +#if defined AC_APPLE_UNIVERSAL_BUILD +# if defined __BIG_ENDIAN__ +# define WORDS_BIGENDIAN 1 +# endif +#else +# ifndef WORDS_BIGENDIAN +/* # undef WORDS_BIGENDIAN */ +# endif +#endif + +/* Define to the type of arg 3 for `accept'. */ +#define ber_socklen_t socklen_t + +/* Define to `char *' if does not define. */ +/* #undef caddr_t */ + +/* Define to empty if `const' does not conform to ANSI C. */ +/* #undef const */ + +/* Define to `int' if doesn't define. */ +/* #undef gid_t */ + +/* Define to `int' if does not define. */ +/* #undef mode_t */ + +/* Define to `long' if does not define. */ +/* #undef off_t */ + +/* Define to `int' if does not define. */ +/* #undef pid_t */ + +/* Define to `int' if does not define. */ +/* #undef sig_atomic_t */ + +/* Define to `unsigned' if does not define. */ +/* #undef size_t */ + +/* define to snprintf routine */ +/* #undef snprintf */ + +/* Define like ber_socklen_t if does not define. */ +/* #undef socklen_t */ + +/* Define to `signed int' if does not define. */ +/* #undef ssize_t */ + +/* Define to `int' if doesn't define. */ +/* #undef uid_t */ + +/* define as empty if volatile is not supported */ +/* #undef volatile */ + +/* define to snprintf routine */ +/* #undef vsnprintf */ + + +/* begin of portable.h.post */ + +#ifdef _WIN32 +/* don't suck in all of the win32 api */ +# define WIN32_LEAN_AND_MEAN 1 +#endif + +#ifndef LDAP_NEEDS_PROTOTYPES +/* force LDAP_P to always include prototypes */ +#define LDAP_NEEDS_PROTOTYPES 1 +#endif + +#ifndef LDAP_REL_ENG +#if (LDAP_VENDOR_VERSION == 000000) && !defined(LDAP_DEVEL) +#define LDAP_DEVEL +#endif +#if defined(LDAP_DEVEL) && !defined(LDAP_TEST) +#define LDAP_TEST +#endif +#endif + +#ifdef HAVE_STDDEF_H +# include +#endif + +#ifdef HAVE_EBCDIC +/* ASCII/EBCDIC converting replacements for stdio funcs + * vsnprintf and snprintf are used too, but they are already + * checked by the configure script + */ +#define fputs ber_pvt_fputs +#define fgets ber_pvt_fgets +#define printf ber_pvt_printf +#define fprintf ber_pvt_fprintf +#define vfprintf ber_pvt_vfprintf +#define vsprintf ber_pvt_vsprintf +#endif + +#include "ac/fdset.h" + +#include "ldap_cdefs.h" +#include "ldap_features.h" + +#include "ac/assert.h" +#include "ac/localize.h" + +#endif /* _LDAP_PORTABLE_H */ +/* end of portable.h.post */ + diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index e7ff1f548e3..e7fd7533fff 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -228,6 +228,11 @@ if(HAVE_AUXV_GETAUXVAL) add_definitions(-DROCKSDB_AUXV_GETAUXVAL_PRESENT) endif() +check_cxx_symbol_exists(elf_aux_info sys/auxv.h HAVE_ELF_AUX_INFO) +if(HAVE_ELF_AUX_INFO) + add_definitions(-DROCKSDB_AUXV_GETAUXVAL_PRESENT) +endif() + include_directories(${ROCKSDB_SOURCE_DIR}) include_directories("${ROCKSDB_SOURCE_DIR}/include") if(WITH_FOLLY_DISTRIBUTED_MUTEX) From 1dda59668918e894b773a6595b9738ab326a1b99 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 10 Oct 2021 17:32:01 +0300 Subject: [PATCH 369/950] Fix releasing query ID and session ID at the end of query processing. --- src/Interpreters/Session.cpp | 9 +++++++++ src/Interpreters/Session.h | 3 +++ src/Server/GRPCServer.cpp | 17 ++++++++++++++++- tests/integration/test_grpc_protocol/test.py | 2 +- 4 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 128df040c7a..5f1b43fe1e8 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -481,5 +481,14 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t return query_context; } + +void Session::releaseSessionID() +{ + if (!named_session) + return; + named_session->release(); + named_session = nullptr; +} + } diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 772ccba7766..ab269bb619c 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -68,6 +68,9 @@ public: ContextMutablePtr makeQueryContext(const ClientInfo & query_client_info) const; ContextMutablePtr makeQueryContext(ClientInfo && query_client_info) const; + /// Releases the currently used session ID so it becomes available for reuse by another session. + void releaseSessionID(); + private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 0fb9d82aca6..7aa1ca06990 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -584,6 +584,7 @@ namespace void finishQuery(); void onException(const Exception & exception); void onFatalError(); + void releaseQueryIDAndSessionID(); void close(); void readQueryInfo(); @@ -1175,6 +1176,7 @@ namespace addProgressToResult(); query_scope->logPeakMemoryUsage(); addLogsToResult(); + releaseQueryIDAndSessionID(); sendResult(); close(); @@ -1205,6 +1207,8 @@ namespace LOG_WARNING(log, "Couldn't send logs to client"); } + releaseQueryIDAndSessionID(); + try { sendException(exception); @@ -1224,7 +1228,7 @@ namespace { try { - finalize = true; + result.mutable_exception()->set_name("FatalError"); addLogsToResult(); sendResult(); } @@ -1234,6 +1238,17 @@ namespace } } + void Call::releaseQueryIDAndSessionID() + { + /// releaseQueryIDAndSessionID() should be called before sending the final result to the client + /// because the client may decide to send another query with the same query ID or session ID + /// immediately after it receives our final result, and it's prohibited to have + /// two queries executed at the same time with the same query ID or session ID. + io.process_list_entry.reset(); + if (session) + session->releaseSessionID(); + } + void Call::close() { responder.reset(); diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index 79879c13c9d..7b2cdee8d76 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -211,7 +211,7 @@ def test_errors_handling(): assert "Table default.t already exists" in e.display_text def test_authentication(): - query("CREATE USER john IDENTIFIED BY 'qwe123'") + query("CREATE USER OR REPLACE john IDENTIFIED BY 'qwe123'") assert query("SELECT currentUser()", user_name="john", password="qwe123") == "john\n" def test_logs(): From 045704274cc1d7042a76608418d29cbc9c36929b Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 10 Oct 2021 16:31:32 +0000 Subject: [PATCH 370/950] implement directory watcher --- .gitmodules | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Storages/FileLog/DirectoryWatcherBase.cpp | 134 ++++++++++++++++++ src/Storages/FileLog/DirectoryWatcherBase.h | 100 +++++++++++++ .../FileLog/FileLogDirectoryWatcher.cpp | 39 +++-- .../FileLog/FileLogDirectoryWatcher.h | 25 ++-- src/Storages/FileLog/StorageFileLog.cpp | 10 +- src/Storages/FileLog/StorageFileLog.h | 4 - 8 files changed, 270 insertions(+), 45 deletions(-) create mode 100644 src/Storages/FileLog/DirectoryWatcherBase.cpp create mode 100644 src/Storages/FileLog/DirectoryWatcherBase.h diff --git a/.gitmodules b/.gitmodules index 7f5c60d2b0c..74d1049ce01 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,6 +1,6 @@ [submodule "contrib/poco"] path = contrib/poco - url = https://github.com/ucasfl/poco.git + url = https://github.com/ClickHouse-Extras/poco.git branch = clickhouse [submodule "contrib/zstd"] path = contrib/zstd diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ee456f81bcf..a8c3ca62e30 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,6 +589,7 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(624, NOT_REGULAR_FILE) \ + M(625, IO_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp new file mode 100644 index 00000000000..ed263767ad2 --- /dev/null +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -0,0 +1,134 @@ +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; + extern const int DIRECTORY_DOESNT_EXIST; + extern const int IO_ERROR; +} + +DirectoryWatcherBase::DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_) + : owner(owner_), path(path_), event_mask(event_mask_) +{ + if (!std::filesystem::exists(path)) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist.", path); + + if (!std::filesystem::is_directory(path)) + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory.", path); + + fd = inotify_init(); + if (fd == -1) + throw Exception("cannot initialize inotify", ErrorCodes::IO_ERROR); + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); + start(); +} + +void DirectoryWatcherBase::watchFunc() +{ + int mask = 0; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_ADDED) + mask |= IN_CREATE; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_REMOVED) + mask |= IN_DELETE; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_MODIFIED) + mask |= IN_MODIFY; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_FROM) + mask |= IN_MOVED_FROM; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_TO) + mask |= IN_MOVED_TO; + + int wd = inotify_add_watch(fd, path.c_str(), mask); + if (wd == -1) + { + owner.onError(Exception(ErrorCodes::IO_ERROR, "Watch directory {} failed.", path)); + } + + std::string buffer; + buffer.resize(4096); + fd_set fds; + while (true) + { + FD_ZERO(&fds); + FD_SET(fd, &fds); + + struct timeval tv; + tv.tv_sec = 0; + tv.tv_usec = 200000; + + if (select(fd + 1, &fds, nullptr, nullptr, &tv) == 1) + { + int n = read(fd, buffer.data(), buffer.size()); + int i = 0; + if (n > 0) + { + while (n > 0) + { + struct inotify_event * p_event = reinterpret_cast(buffer.data() + i); + + if (p_event->len > 0) + { + if ((p_event->mask & IN_CREATE) && (eventMask() & DirectoryWatcherBase::DW_ITEM_ADDED)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_ADDED); + owner.onItemAdded(ev); + } + if ((p_event->mask & IN_DELETE) && (eventMask() & DirectoryWatcherBase::DW_ITEM_REMOVED)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_REMOVED); + owner.onItemRemoved(ev); + } + if ((p_event->mask & IN_MODIFY) && (eventMask() & DirectoryWatcherBase::DW_ITEM_MODIFIED)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_MODIFIED); + owner.onItemModified(ev); + } + if ((p_event->mask & IN_MOVED_FROM) && (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_FROM)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_MOVED_FROM); + owner.onItemMovedFrom(ev); + } + if ((p_event->mask & IN_MOVED_TO) && (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_TO)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_MOVED_TO); + owner.onItemMovedTo(ev); + } + } + + i += sizeof(inotify_event) + p_event->len; + n -= sizeof(inotify_event) + p_event->len; + } + } + } + } +} + + +DirectoryWatcherBase::~DirectoryWatcherBase() +{ + stop(); + close(fd); +} + +void DirectoryWatcherBase::start() +{ + if (watch_task) + watch_task->activateAndSchedule(); +} + + +void DirectoryWatcherBase::stop() +{ + if (watch_task) + watch_task->deactivate(); +} + +} diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h new file mode 100644 index 00000000000..53fb3862a41 --- /dev/null +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -0,0 +1,100 @@ +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ +class FileLogDirectoryWatcher; + +class DirectoryWatcherBase : WithContext +{ + /// This class is used to get notifications about changes + /// to the filesystem, more specifically, to a specific + /// directory. Changes to a directory are reported via + /// events. + /// + /// A thread will be created that watches the specified + /// directory for changes. Events are reported in the context + /// of this thread. + /// + /// Note that changes to files in subdirectories of the watched + /// directory are not reported. Separate DirectoryWatcher objects + /// must be created for these directories if they should be watched. +public: + enum DirectoryEventType + { + DW_ITEM_ADDED = 1, + /// A new item has been created and added to the directory. + + DW_ITEM_REMOVED = 2, + /// An item has been removed from the directory. + + DW_ITEM_MODIFIED = 4, + /// An item has been modified. + + DW_ITEM_MOVED_FROM = 8, + /// An item has been renamed or moved. This event delivers the old name. + + DW_ITEM_MOVED_TO = 16 + /// An item has been renamed or moved. This event delivers the new name. + }; + + enum DirectoryEventMask + { + /// Enables all event types. + DW_FILTER_ENABLE_ALL = 31, + + /// Disables all event types. + DW_FILTER_DISABLE_ALL = 0 + }; + + struct DirectoryEvent + { + DirectoryEvent(const std::string & f, DirectoryEventType ev) : path(f), event(ev) { } + + /// The directory or file that has been changed. + const std::string & path; + /// The kind of event. + DirectoryEventType event; + }; + + + DirectoryWatcherBase() = delete; + DirectoryWatcherBase(const DirectoryWatcherBase &) = delete; + DirectoryWatcherBase & operator=(const DirectoryWatcherBase &) = delete; + + /// Creates a DirectoryWatcher for the directory given in path. + /// To enable only specific events, an eventMask can be specified by + /// OR-ing the desired event IDs (e.g., DW_ITEM_ADDED | DW_ITEM_MODIFIED). + explicit DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_ = DW_FILTER_ENABLE_ALL); + + ~DirectoryWatcherBase(); + + /// Returns the value of the eventMask passed to the constructor. + int eventMask() const { return event_mask; } + + /// Returns the directory being watched. + const std::string & directory() const; + + [[noreturn]] void watchFunc(); + +protected: + void start(); + void stop(); + +private: + FileLogDirectoryWatcher & owner; + + using TaskThread = BackgroundSchedulePool::TaskHolder; + TaskThread watch_task; + + const std::string path; + int event_mask; + int fd; +}; + +} diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 3b4f89d50f5..b0fbf5d535b 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -1,18 +1,10 @@ #include -#include -#include -FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_unique(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) +namespace DB +{ +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) + : path(path_), dw(std::make_unique(*this, path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { - /// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported on Linux. - /// On other platforms, a file rename or move operation will be reported via a - /// DW_ITEM_REMOVED and a DW_ITEM_ADDED event. The order of these two events is not defined. - dw->itemAdded += Poco::delegate(this, &FileLogDirectoryWatcher::onItemAdded); - dw->itemRemoved += Poco::delegate(this, &FileLogDirectoryWatcher::onItemRemoved); - dw->itemModified += Poco::delegate(this, &FileLogDirectoryWatcher::onItemModified); - dw->itemMovedFrom += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedFrom); - dw->itemMovedTo += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedTo); } FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset() @@ -36,12 +28,12 @@ const std::string & FileLogDirectoryWatcher::getPath() const return path; } -void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemAdded"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -54,12 +46,12 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director } -void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemRemoved"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -77,11 +69,11 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct /// So, if we record all of these events, it will use a lot of memory, and then we /// need to handle it one by one in StorageFileLog::updateFileInfos, this is unnecessary /// because it is equal to just record and handle one MODIY event -void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); - auto event_path = ev.item.path(); + auto event_path = ev.path; EventInfo info{ev.event, "onItemModified"}; /// Already have MODIFY event for this file if (auto it = events.find(event_path); it != events.end()) @@ -97,12 +89,12 @@ void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::Direc } } -void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedFrom"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -114,12 +106,12 @@ void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::Dire } } -void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedTo"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -131,10 +123,11 @@ void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::Direct } } -void FileLogDirectoryWatcher::onError(const Poco::Exception & e) +void FileLogDirectoryWatcher::onError(const Exception & e) { std::lock_guard lock(mutex); LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); error.has_error = true; error.error_msg = e.message(); } +} diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 00ee7ddfc40..c7216fa4872 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -1,20 +1,21 @@ #pragma once -#include -#include -#include +#include #include #include #include +namespace DB +{ + class FileLogDirectoryWatcher { public: struct EventInfo { - Poco::DirectoryWatcher::DirectoryEventType type; + DirectoryWatcherBase::DirectoryEventType type; std::string callback; }; @@ -35,13 +36,12 @@ public: const std::string & getPath() const; -protected: - void onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent & ev); - void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent & ev); - void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent & ev); - void onError(const Poco::Exception &); + void onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev); + void onError(const Exception &); private: const std::string path; @@ -53,7 +53,7 @@ private: /// running, it may access events during events destruction, leads to data race. Events events; - std::unique_ptr dw; + std::unique_ptr dw; Poco::Logger * log; @@ -61,3 +61,4 @@ private: Error error; }; +} diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a75c9b0190e..18515304c0c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -798,7 +798,7 @@ bool StorageFileLog::updateFileInfos() { switch (event_info.type) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: + case DirectoryWatcherBase::DW_ITEM_ADDED: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); /// Check if it is a regular file, and new file may be renamed or removed @@ -821,7 +821,7 @@ bool StorageFileLog::updateFileInfos() break; } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + case DirectoryWatcherBase::DW_ITEM_MODIFIED: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); /// When new file added and appended, it has two event: DW_ITEM_ADDED @@ -833,15 +833,15 @@ bool StorageFileLog::updateFileInfos() break; } - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + case DirectoryWatcherBase::DW_ITEM_REMOVED: + case DirectoryWatcherBase::DW_ITEM_MOVED_FROM: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) it->second.status = FileStatus::REMOVED; break; } - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case DirectoryWatcherBase::DW_ITEM_MOVED_TO: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f3b85d47313..1b1d5f6671a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -8,8 +8,6 @@ #include #include -#include -#include #include #include @@ -179,8 +177,6 @@ private: /// later select should forbid to execute. std::atomic running_streams = 0; - using TaskThread = BackgroundSchedulePool::TaskHolder; - void loadFiles(); void loadMetaFiles(bool attach); From 934b72ddd14dd4e3f01f126c452af238fba33d48 Mon Sep 17 00:00:00 2001 From: mikael Date: Sun, 10 Oct 2021 18:54:38 +0200 Subject: [PATCH 371/950] Add FreeBSD aarch64 bits for cross-builds for CI. --- cmake/freebsd/toolchain-aarch64.cmake | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 cmake/freebsd/toolchain-aarch64.cmake diff --git a/cmake/freebsd/toolchain-aarch64.cmake b/cmake/freebsd/toolchain-aarch64.cmake new file mode 100644 index 00000000000..b8fdb4bbb7c --- /dev/null +++ b/cmake/freebsd/toolchain-aarch64.cmake @@ -0,0 +1,22 @@ +set (CMAKE_SYSTEM_NAME "FreeBSD") +set (CMAKE_SYSTEM_PROCESSOR "aarch64") +set (CMAKE_C_COMPILER_TARGET "aarch64-unknown-freebsd12") +set (CMAKE_CXX_COMPILER_TARGET "aarch64-unknown-freebsd12") +set (CMAKE_ASM_COMPILER_TARGET "aarch64-unknown-freebsd12") +set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../toolchain/freebsd-aarch64") + +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) # disable linkage check - it doesn't work in CMake + +set (CMAKE_AR "/usr/bin/ar" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "/usr/bin/ranlib" CACHE FILEPATH "" FORCE) + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From af4066c255149d1ab1cef9b7fba0c9c47d1bb265 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Oct 2021 21:54:15 +0300 Subject: [PATCH 372/950] Remove printf --- base/daemon/SentryWriter.cpp | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index efd915b1e5b..ac771b9bf47 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "Common/config_version.h" @@ -163,34 +164,34 @@ void SentryWriter::onFault(int sig, const std::string & error_message, const Sta if (stack_size > 0) { ssize_t offset = stack_trace.getOffset(); - char instruction_addr[100]; + + char instruction_addr[19] + { + '0', 'x', + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f', + '\0' + }; + StackTrace::Frames frames; StackTrace::symbolize(stack_trace.getFramePointers(), offset, stack_size, frames); + for (ssize_t i = stack_size - 1; i >= offset; --i) { const StackTrace::Frame & current_frame = frames[i]; sentry_value_t sentry_frame = sentry_value_new_object(); UInt64 frame_ptr = reinterpret_cast(current_frame.virtual_addr); - if (std::snprintf(instruction_addr, sizeof(instruction_addr), "0x%" PRIx64, frame_ptr) >= 0) - { - sentry_value_set_by_key(sentry_frame, "instruction_addr", sentry_value_new_string(instruction_addr)); - } + writeHexUIntLowercase(frame_ptr, instruction_addr + 2); + sentry_value_set_by_key(sentry_frame, "instruction_addr", sentry_value_new_string(instruction_addr)); if (current_frame.symbol.has_value()) - { sentry_value_set_by_key(sentry_frame, "function", sentry_value_new_string(current_frame.symbol.value().c_str())); - } if (current_frame.file.has_value()) - { sentry_value_set_by_key(sentry_frame, "filename", sentry_value_new_string(current_frame.file.value().c_str())); - } if (current_frame.line.has_value()) - { sentry_value_set_by_key(sentry_frame, "lineno", sentry_value_new_int32(current_frame.line.value())); - } sentry_value_append(sentry_frames, sentry_frame); } From 6c33eaee325e71bcf582f255321aafc7d2a7aa76 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 22:23:05 +0300 Subject: [PATCH 373/950] Fix filtering by tuple (some conditions was lost during analyzing) Fixes: #29281 Fixes: test_cluster_copier/test_three_nodes.py::test --- src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- tests/queries/0_stateless/02030_tuple_filter.reference | 1 - tests/queries/0_stateless/02030_tuple_filter.sql | 7 ++++++- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index b0071d481d6..a32eecd4a49 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -158,7 +158,7 @@ bool MergeTreeWhereOptimizer::tryAnalyzeTuple(Conditions & res, const ASTFunctio else if (const auto * child_ident = child->as()) fetch_sign_column = std::make_shared(child_ident->name()); else - continue; + return false; ASTPtr fetch_sign_value = std::make_shared(tuple_lit.at(i)); ASTPtr func_node = makeASTFunction("equals", fetch_sign_column, fetch_sign_value); diff --git a/tests/queries/0_stateless/02030_tuple_filter.reference b/tests/queries/0_stateless/02030_tuple_filter.reference index b7bd4c157f0..a2e1290c0e1 100644 --- a/tests/queries/0_stateless/02030_tuple_filter.reference +++ b/tests/queries/0_stateless/02030_tuple_filter.reference @@ -6,4 +6,3 @@ 1 A 2021-01-01 1 A 2021-01-01 1 A 2021-01-01 -1 A 2021-01-01 diff --git a/tests/queries/0_stateless/02030_tuple_filter.sql b/tests/queries/0_stateless/02030_tuple_filter.sql index 606955adcb7..5efedeb8c0d 100644 --- a/tests/queries/0_stateless/02030_tuple_filter.sql +++ b/tests/queries/0_stateless/02030_tuple_filter.sql @@ -11,7 +11,12 @@ SELECT * FROM test_tuple_filter WHERE (1, 'A') = (id, value); SELECT * FROM test_tuple_filter WHERE (id, value) = (1, 'A') AND (id, log_date) = (1, '2021-01-01'); SELECT * FROM test_tuple_filter WHERE ((id, value), id * 2) = ((1, 'A'), 2); SELECT * FROM test_tuple_filter WHERE ((id, value), log_date) = ((1, 'A'), '2021-01-01'); -SELECT * FROM test_tuple_filter WHERE (1, (1, (1, (1, (id, value))))) = (1, (1, (1, (1, (1, 'A'))))); + +-- not supported functions (concat) do not lost +SELECT * FROM test_tuple_filter WHERE (id, value, value||'foo') = ('1', 'A', 'A'); + +-- Condition fully moved to PREWHERE and such conditions does not supported yet. +SELECT * FROM test_tuple_filter WHERE (1, (1, (1, (1, (id, value))))) = (1, (1, (1, (1, (1, 'A'))))); -- { serverError INDEX_NOT_USED } -- not implemented yet SELECT * FROM test_tuple_filter WHERE (1, value) = (id, 'A'); -- { serverError INDEX_NOT_USED } From b83655b348f93cc4f720f61921d57d59f19f0dc8 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 10 Oct 2021 19:38:53 +0000 Subject: [PATCH 374/950] ru translation --- .../functions/other-functions.md | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 31e81b04330..c91d5caef8d 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2313,3 +2313,65 @@ SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0 └─────────┘ ``` +## shardNum {#shard-num} + +Возвращает индекс шарда, который обрабатывает часть данных распределенного запроса. Индексы начинаются с `1`. +Если запрос не распределенный, то возвращается константное значение `0`. + +**Синтаксис** + +``` sql +shardNum() +``` + +**Возвращаемое значение** + +- индекс шарда или константа `0`. + +Тип: [UInt32](../../sql-reference/data-types/int-uint.md). + +**Пример** + +В примере ниже используется конфигурация с двумя шардами. На каждом шарде выполняется запрос к таблице [system.one](../../operations/system-tables/one.md). + +Запрос: + +``` sql +CREATE TABLE shard_num_example (dummy UInt8) + ENGINE=Distributed(test_cluster_two_shards_localhost, system, one, dummy); +SELECT dummy, shardNum(), shardCount() FROM shard_num_example; +``` + +Результат: + +``` text +┌─dummy─┬─shardNum()─┬─shardCount()─┐ +│ 0 │ 2 │ 2 │ +│ 0 │ 1 │ 2 │ +└───────┴────────────┴──────────────┘ +``` + +**См. также** + +- [Distributed Table Engine](../../engines/table-engines/special/distributed.md) + +## shardCount {#shard-count} + +Возвращает общее количество шардов для распределенного запроса. +Если запрос не распределенный, то возвращается константное значение `0`. + +**Синтаксис** + +``` sql +shardCount() +``` + +**Возвращаемое значение** + +- Общее количество шардов или `0`. + +Тип: [UInt32](../../sql-reference/data-types/int-uint.md). + +**См. также** + +- Пример использования функции [shardNum()](#shard-num) также содержит вызов `shardCount()`. From e229847862dfa708a9dd2f06fc3c10372a684681 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Sun, 10 Oct 2021 14:42:06 -0600 Subject: [PATCH 375/950] Documentation correction for units for block size, plus some additional context for storage system recommendations --- docs/en/operations/tips.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index f7199372c80..27fc7d47a76 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -60,7 +60,7 @@ $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size Calculate the exact number from the number of devices and the block size, using the formula: `2 * num_devices * chunk_size_in_bytes / 4096`. -A block size of 1024 KB is sufficient for all RAID configurations. +A block size of 1024 bytes (1KB) is sufficient for most RAID configurations. A block size range of 1KB to 16KB is common across many storage system vendors or DBMS. Never set the block size too small or too large. You can use RAID-0 on SSD. From f37dac03f46790317e4ea5957ff64056ea189beb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 01:00:47 +0300 Subject: [PATCH 376/950] Fix lock-order-inversion between DROP TABLE for DatabaseMemory and LiveView CI stress founds [1], TSan report: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=509) Cycle in lock order graph: M71436201 (0x7b5800a91308) => M237700395169415472 (0x000000000000) => M71436201 Mutex M237700395169415472 acquired here while holding mutex M71436201 in thread T723: 3 std::__1::lock_guard::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse+0x154b0db9) 4 DB::DatabaseWithOwnTablesBase::tryGetTable() const obj-x86_64-linux-gnu/../src/Databases/DatabasesCommon.cpp:37:21 (clickhouse+0x154b0db9) 5 DB::DatabaseMemory::tryGetTableUUID() const obj-x86_64-linux-gnu/../src/Databases/DatabaseMemory.cpp:95:22 (clickhouse+0x15466bb5) 6 DB::Context::resolveStorageID() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2672:90 (clickhouse+0x155e6aa1) 7 DB::JoinedTables::getLeftTableStorage() obj-x86_64-linux-gnu/../src/Interpreters/JoinedTables.cpp:200:29 (clickhouse+0x15eee962) 8 DB::InterpreterSelectQuery::InterpreterSelectQuery() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:321:33 (clickhouse+0x15b792be) 9 DB::InterpreterSelectQuery::InterpreterSelectQuery() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:160:7 (clickhouse+0x15b78160) 10 DB::StorageLiveView::collectMergeableBlocks() obj-x86_64-linux-gnu/../src/Storages/LiveView/StorageLiveView.cpp:113:28 (clickhouse+0x16aae192) 11 DB::StorageLiveView::getNewBlocks() obj-x86_64-linux-gnu/../src/Storages/LiveView/StorageLiveView.cpp:384:33 (clickhouse+0x16ab393a) 12 DB::StorageLiveView::refresh(bool) obj-x86_64-linux-gnu/../src/Storages/LiveView/StorageLiveView.cpp:528:13 (clickhouse+0x16ab6395) 13 DB::StorageLiveView::read() obj-x86_64-linux-gnu/../src/Storages/LiveView/StorageLiveView.cpp:545:9 (clickhouse+0x16ab6395) 21 DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr, bool, DB::QueryProcessingStage::Enum) obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:950:30 (clickhouse+0x16101441) Hint: use TSAN_OPTIONS=second_deadlock_stack=1 to get more informative warning message Mutex M71436201 acquired here while holding mutex M237700395169415472 in thread T723: 3 std::__1::lock_guard::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse+0x16ab5c29) 4 DB::StorageLiveView::drop() obj-x86_64-linux-gnu/../src/Storages/LiveView/StorageLiveView.cpp:477:21 (clickhouse+0x16ab5c29) 5 DB::DatabaseMemory::dropTable() obj-x86_64-linux-gnu/../src/Databases/DatabaseMemory.cpp:44:16 (clickhouse+0x15465cd6) 8 DB::InterpreterDropQuery::execute() obj-x86_64-linux-gnu/../src/Interpreters/InterpreterDropQuery.cpp:62:16 (clickhouse+0x15afa679) [1]: https://clickhouse-test-reports.s3.yandex.net/29856/42ca2b4bb241827edf69bbd6938d6b19c31935f1/stress_test_(thread).html#fail1 v2: fix attachTableUnlocked() (as pointed by @tavplubix) v3: use SCOPE_EXIT() and so avoid holding a lock during removing from fs --- src/Databases/DatabaseMemory.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 8f909d280c5..ff4021cf6d2 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -41,7 +42,14 @@ void DatabaseMemory::dropTable( auto table = detachTableUnlocked(table_name, lock); try { + /// Remove table w/o lock since: + /// - it does not require it + /// - it may cause lock-order-inversion if underlying storage need to + /// resolve tables (like StorageLiveView) + SCOPE_EXIT(lock.lock()); + lock.unlock(); table->drop(); + if (table->storesDataOnDisk()) { assert(database_name != DatabaseCatalog::TEMPORARY_DATABASE); From 07aea95a755ab5c5fdfe9123f56f2e1bdd435b87 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Oct 2021 00:00:12 +0300 Subject: [PATCH 377/950] Fix rabbitmq tests. --- .../integration/test_storage_rabbitmq/test.py | 55 +------------------ 1 file changed, 2 insertions(+), 53 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 696294f4bde..36d63588386 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -65,7 +65,8 @@ def rabbitmq_cluster(): def rabbitmq_setup_teardown(): print("RabbitMQ is available - running test") yield # run test - instance.query('DROP TABLE IF EXISTS test.rabbitmq') + for table_name in ['view', 'consumer', 'rabbitmq']: + instance.query(f'DROP TABLE IF EXISTS test.{table_name}') # Tests @@ -195,8 +196,6 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -266,8 +265,6 @@ def test_rabbitmq_macros(rabbitmq_cluster): def test_rabbitmq_materialized_view(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -297,19 +294,12 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): if (rabbitmq_check_result(result)): break - instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; - ''') - connection.close() rabbitmq_check_result(result, True) def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -339,11 +329,6 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): if rabbitmq_check_result(result): break - instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; - ''') - connection.close() rabbitmq_check_result(result, True) @@ -404,8 +389,6 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): @pytest.mark.skip(reason="clichouse_path with rabbitmq.proto fails to be exported") def test_rabbitmq_protobuf(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq (key UInt64, value String) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -457,11 +440,6 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): if rabbitmq_check_result(result): break - instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; - ''') - rabbitmq_check_result(result, True) @@ -477,8 +455,6 @@ def test_rabbitmq_big_message(rabbitmq_cluster): channel = connection.channel() instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq (key UInt64, value String) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -500,10 +476,6 @@ def test_rabbitmq_big_message(rabbitmq_cluster): break connection.close() - instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; - ''') assert int(result) == rabbitmq_messages * batch_messages, 'ClickHouse lost some messages: {}'.format(result) @@ -521,8 +493,6 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): rabbitmq_num_consumers = 10, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view (key UInt64, value UInt64, channel_id String) ENGINE = MergeTree ORDER BY key @@ -1398,7 +1368,6 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): def test_rabbitmq_virtual_columns(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; CREATE TABLE test.rabbitmq_virtuals (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -1459,8 +1428,6 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.rabbitmq_virtuals_mv (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -1606,8 +1573,6 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): instance.query(''' DROP TABLE IF EXISTS test.consume; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; @@ -1665,8 +1630,6 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): break instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; DROP TABLE test.consume; DROP TABLE test.producer_reconnect; ''') @@ -1703,8 +1666,6 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): properties=pika.BasicProperties(delivery_mode=2, message_id=str(msg_id))) connection.close() instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; @@ -1743,9 +1704,6 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; - DROP TABLE IF EXISTS test.rabbitmq; CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', @@ -1834,8 +1792,6 @@ def test_rabbitmq_no_connection_at_startup_2(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_num_consumers = '5', rabbitmq_row_delimiter = '\\n'; - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; @@ -1899,8 +1855,6 @@ def test_rabbitmq_format_factory_settings(rabbitmq_cluster): break; instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view ( id String, date DateTime ) ENGINE = MergeTree ORDER BY id; @@ -2002,8 +1956,6 @@ def test_rabbitmq_queue_settings(rabbitmq_cluster): connection.close() instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS @@ -2056,9 +2008,6 @@ def test_rabbitmq_queue_consume(rabbitmq_cluster): rabbitmq_format = 'JSONEachRow', rabbitmq_queue_base = 'rabbit_queue', rabbitmq_queue_consume = 1; - - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS From 341eb1f9bfb1a60e19f1159b9e7f1822ed7010c4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 11 Oct 2021 00:05:53 +0300 Subject: [PATCH 378/950] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 20bf9a10986..a6df18b323c 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -162,5 +162,6 @@ toc_title: Adopters | Zagrava Trading | — | — | — | — | [Job offer, May 2021](https://twitter.com/datastackjobs/status/1394707267082063874) | | Beeline | Telecom | Data Platform | — | — | [Blog post, July 2021](https://habr.com/en/company/beeline/blog/567508/) | | Ecommpay | Payment Processing | Logs | — | — | [Video, Nov 2019](https://www.youtube.com/watch?v=d3GdZTOWGLk) | +| Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, Oct 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From cb8a66e5155b0e4c1be0a037db8c54f48d47372c Mon Sep 17 00:00:00 2001 From: olgarev Date: Sun, 10 Oct 2021 23:34:26 +0000 Subject: [PATCH 379/950] Settings and links --- docs/en/operations/settings/settings.md | 38 ++++++++++++++++++- .../statements/select/prewhere.md | 10 +++-- docs/ru/operations/settings/settings.md | 38 ++++++++++++++++++- .../statements/select/prewhere.md | 12 +++--- 4 files changed, 87 insertions(+), 11 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f78fbc8a2bc..0491674b701 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3783,4 +3783,40 @@ Result: │ 20 │ 20 │ 10 │ │ 10 │ 20 │ 30 │ └─────┴─────┴───────┘ -``` \ No newline at end of file +``` + +## optimize_move_to_prewhere {#optimize_move_to_prewhere} + +Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries. + +Works only for [*MergeTree](../../engines/table-engines/mergetree-family/index.md) tables. + +Possible values: + +- 0 — Automatic `PREWHERE` optimization is disabled. +- 1 — Automatic `PREWHERE` optimization is enabled. + +Default value: `1`. + +**See Also** + +- [PREWHERE](../../sql-reference/statements/select/prewhere.md) clause in `SELECT` queries + +## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} + +Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. + +Works only for [*MergeTree](../../engines/table-engines/mergetree-family/index.md) tables. + +Possible values: + +- 0 — Automatic `PREWHERE` optimization in `SELECT` queries with `FINAL` modifier is disabled. +- 1 — Automatic `PREWHERE` optimization in `SELECT` queries with `FINAL` modifier is enabled. + +Default value: `0`. + +**See Also** + +- [PREWHERE](../../sql-reference/statements/select/prewhere.md) clause in `SELECT` queries +- [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier in `SELECT` queries +- [optimize_move_to_prewhere](#optimize_move_to_prewhere) setting \ No newline at end of file diff --git a/docs/en/sql-reference/statements/select/prewhere.md b/docs/en/sql-reference/statements/select/prewhere.md index ada8fff7012..646bb83e692 100644 --- a/docs/en/sql-reference/statements/select/prewhere.md +++ b/docs/en/sql-reference/statements/select/prewhere.md @@ -6,7 +6,7 @@ toc_title: PREWHERE Prewhere is an optimization to apply filtering more efficiently. It is enabled by default even if `PREWHERE` clause is not specified explicitly. It works by automatically moving part of [WHERE](../../../sql-reference/statements/select/where.md) condition to prewhere stage. The role of `PREWHERE` clause is only to control this optimization if you think that you know how to do it better than it happens by default. -With prewhere optimization, at first only the columns necessary for executing prewhere expression are read. Then the other columns are read that are needed for running the rest of the query, but only those blocks where the prewhere expression is “true” at least for some rows. If there are a lot of blocks where prewhere expression is “false” for all rows and prewhere needs less columns than other parts of query, this often allows to read a lot less data from disk for query execution. +With prewhere optimization, at first only the columns necessary for executing prewhere expression are read. Then the other columns are read that are needed for running the rest of the query, but only those blocks where the prewhere expression is `true` at least for some rows. If there are a lot of blocks where prewhere expression is `false` for all rows and prewhere needs less columns than other parts of query, this often allows to read a lot less data from disk for query execution. ## Controlling Prewhere Manually {#controlling-prewhere-manually} @@ -14,11 +14,13 @@ The clause has the same meaning as the `WHERE` clause. The difference is in whic A query may simultaneously specify `PREWHERE` and `WHERE`. In this case, `PREWHERE` precedes `WHERE`. -If the `optimize_move_to_prewhere` setting is set to 0, heuristics to automatically move parts of expressions from `WHERE` to `PREWHERE` are disabled. +If the [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) setting is set to 0, heuristics to automatically move parts of expressions from `WHERE` to `PREWHERE` are disabled. + +If query has [FINAL](from.md#select-from-final) modifier, the `PREWHERE` optimization is not always correct. It is enabled only if both settings [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) and [optimize_move_to_prewhere_if_final](../../../operations/settings/settings.md#optimize_move_to_prewhere_if_final) are turned on. !!! note "Attention" - The `PREWHERE` section is executed before` FINAL`, so the results of `FROM FINAL` queries may be skewed when using` PREWHERE` with fields not in the `ORDER BY` section of a table. + The `PREWHERE` section is executed before `FINAL`, so the results of `FROM ... FINAL` queries may be skewed when using `PREWHERE` with fields not in the `ORDER BY` section of a table. ## Limitations {#limitations} -`PREWHERE` is only supported by tables from the `*MergeTree` family. +`PREWHERE` is only supported by tables from the [*MergeTree](../../../engines/table-engines/mergetree-family/index.md) family. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 500485aea2f..887c59c3b09 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3572,4 +3572,40 @@ SELECT * FROM positional_arguments ORDER BY 2,3; │ 20 │ 20 │ 10 │ │ 10 │ 20 │ 30 │ └─────┴─────┴───────┘ -``` \ No newline at end of file +``` + +## optimize_move_to_prewhere {#optimize_move_to_prewhere} + +Включает или отключает автоматическую оптимизацию [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах [SELECT](../../sql-reference/statements/select/index.md). + +Работает только с таблицами семейства [*MergeTree](../../engines/table-engines/mergetree-family/index.md). + +Возможные значения: + +- 0 — автоматическая оптимизация `PREWHERE` отключена. +- 1 — автоматическая оптимизация `PREWHERE` включена. + +Значение по умолчанию: `1`. + +**См. также** + +- секция [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах `SELECT` + +## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} + +Включает или отключает автоматическую оптимизацию [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах [SELECT](../../sql-reference/statements/select/index.md) с модификатором [FINAL](../../sql-reference/statements/select/from.md#select-from-final). + +Работает только с таблицами семейства [*MergeTree](../../engines/table-engines/mergetree-family/index.md). + +Возможные значения: + +- 0 — автоматическая оптимизация `PREWHERE` в запросах `SELECT` с модификатором `FINAL` отключена. +- 1 — автоматическая оптимизация `PREWHERE` в запросах `SELECT` с модификатором `FINAL` включена. + +Значение по умолчанию: `0`. + +**См. также** + +- секция [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах `SELECT` +- модификатор [FINAL](../../sql-reference/statements/select/from.md#select-from-final) в запросах `SELECT` +- настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/select/prewhere.md b/docs/ru/sql-reference/statements/select/prewhere.md index 5ba25e6fa6e..84f8869b41e 100644 --- a/docs/ru/sql-reference/statements/select/prewhere.md +++ b/docs/ru/sql-reference/statements/select/prewhere.md @@ -8,17 +8,19 @@ Prewhere — это оптимизация для более эффективн При оптимизации prewhere сначала читываются только те столбцы, которые необходимы для выполнения выражения prewhere. Затем читаются другие столбцы, необходимые для выполнения остальной части запроса, но только те блоки, в которых находится выражение prewhere «верно» по крайней мере для некоторых рядов. Если есть много блоков, где выражение prewhere «ложно» для всех строк и для выражения prewhere требуется меньше столбцов, чем для других частей запроса, это часто позволяет считывать гораздо меньше данных с диска для выполнения запроса. -## Управление prewhere вручную {#controlling-prewhere-manually} +## Управление PREWHERE вручную {#controlling-prewhere-manually} `PREWHERE` имеет смысл использовать, если есть условия фильтрации, которые использует меньшинство столбцов из тех, что есть в запросе, но достаточно сильно фильтрует данные. Таким образом, сокращается количество читаемых данных. -В запрос может быть одновременно указано и `PREWHERE` и `WHERE`. В этом случае, `PREWHERE` предшествует `WHERE`. +В запросе может быть одновременно указаны и `PREWHERE`, и `WHERE`. В этом случае `PREWHERE` предшествует `WHERE`. -Если значение параметра `optimize_move_to_prewhere` равно 0, эвристика по автоматическому перемещнию части выражений из `WHERE` к `PREWHERE` отключается. +Если значение параметра [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) равно 0, эвристика по автоматическому перемещнию части выражений из `WHERE` к `PREWHERE` отключается. + +Если в запросе есть модификатор [FINAL](from.md#select-from-final), оптимизация `PREWHERE` не всегда корректна. Она действует только если включены обе настройки [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) и [optimize_move_to_prewhere_if_final](../../../operations/settings/settings.md#optimize_move_to_prewhere_if_final). !!! note "Внимание" - Секция `PREWHERE` выполняется до `FINAL`, поэтому результаты запросов `FROM FINAL` могут исказится при использовании `PREWHERE` с полями не входящями в `ORDER BY` таблицы. + Секция `PREWHERE` выполняется до `FINAL`, поэтому результаты запросов `FROM ... FINAL` могут исказиться при использовании `PREWHERE` с полями, не входящями в `ORDER BY` таблицы. ## Ограничения {#limitations} -`PREWHERE` поддерживается только табличными движками из семейства `*MergeTree`. +`PREWHERE` поддерживается только табличными движками из семейства [*MergeTree](../../../engines/table-engines/mergetree-family/index.md). From fa3c13cf81a314158c2fa292f09749b1f6c9dec5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 04:37:04 +0300 Subject: [PATCH 380/950] Experiment with minimized toolchain --- .gitmodules | 3 +++ cmake/linux/toolchain-aarch64.cmake | 18 +++++++++++------- contrib/sysroot | 1 + 3 files changed, 15 insertions(+), 7 deletions(-) create mode 160000 contrib/sysroot diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..696676200fe 100644 --- a/.gitmodules +++ b/.gitmodules @@ -249,3 +249,6 @@ [submodule "contrib/magic_enum"] path = contrib/magic_enum url = https://github.com/Neargye/magic_enum +[submodule "contrib/sysroot"] + path = contrib/sysroot + url = https://github.com/ClickHouse-Extras/sysroot.git diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index b4dc6e45cbb..94e81d13c2e 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -1,17 +1,21 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + set (CMAKE_SYSTEM_NAME "Linux") set (CMAKE_SYSTEM_PROCESSOR "aarch64") set (CMAKE_C_COMPILER_TARGET "aarch64-linux-gnu") set (CMAKE_CXX_COMPILER_TARGET "aarch64-linux-gnu") set (CMAKE_ASM_COMPILER_TARGET "aarch64-linux-gnu") -set (CMAKE_SYSROOT "${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64/aarch64-linux-gnu/libc") -# We don't use compiler from toolchain because it's gcc-8, and we provide support only for gcc-9. -set (CMAKE_AR "${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64/bin/aarch64-linux-gnu-ar" CACHE FILEPATH "" FORCE) -set (CMAKE_RANLIB "${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64/bin/aarch64-linux-gnu-ranlib" CACHE FILEPATH "" FORCE) +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-aarch64") -set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64") -set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64") -set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${CMAKE_CURRENT_LIST_DIR}/../toolchain/linux-aarch64") +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") + +set (CMAKE_AR "llvm-ar" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "llvm-ranlib" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) diff --git a/contrib/sysroot b/contrib/sysroot new file mode 160000 index 00000000000..ae9209a6433 --- /dev/null +++ b/contrib/sysroot @@ -0,0 +1 @@ +Subproject commit ae9209a643374c830a509ca75fd41d89ae189213 From aabe52e3e95b6ad99ca28de4081e2820c249bb5a Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Mon, 11 Oct 2021 10:39:55 +0800 Subject: [PATCH 381/950] Fix bug for quantile fusion --- src/Interpreters/GatherFunctionQuantileVisitor.cpp | 2 +- .../0_stateless/01956_fuse_quantile_optimization.reference | 1 + tests/queries/0_stateless/01956_fuse_quantile_optimization.sql | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/GatherFunctionQuantileVisitor.cpp b/src/Interpreters/GatherFunctionQuantileVisitor.cpp index ec3866ba0c0..3eb474c271c 100644 --- a/src/Interpreters/GatherFunctionQuantileVisitor.cpp +++ b/src/Interpreters/GatherFunctionQuantileVisitor.cpp @@ -47,7 +47,7 @@ void GatherFunctionQuantileData::visit(ASTFunction & function, ASTPtr & ast) void GatherFunctionQuantileData::FuseQuantileAggregatesData::addFuncNode(ASTPtr & ast) { const auto * func = ast->as(); - if (!func) + if (!func || func->parameters == nullptr) return; const auto & arguments = func->arguments->children; diff --git a/tests/queries/0_stateless/01956_fuse_quantile_optimization.reference b/tests/queries/0_stateless/01956_fuse_quantile_optimization.reference index defad422cad..d4cb1477ed6 100644 --- a/tests/queries/0_stateless/01956_fuse_quantile_optimization.reference +++ b/tests/queries/0_stateless/01956_fuse_quantile_optimization.reference @@ -95,3 +95,4 @@ FROM FROM numbers(10) ) GROUP BY b +1 1 1 diff --git a/tests/queries/0_stateless/01956_fuse_quantile_optimization.sql b/tests/queries/0_stateless/01956_fuse_quantile_optimization.sql index 2a97c60882c..a4729e89755 100644 --- a/tests/queries/0_stateless/01956_fuse_quantile_optimization.sql +++ b/tests/queries/0_stateless/01956_fuse_quantile_optimization.sql @@ -71,3 +71,6 @@ SELECT quantileTimingWeighted([[[[['-214748364.8'], NULL]], [[[quantileTimingWei SELECT quantileTimingWeighted([quantileTimingWeighted(0.5)(1, 1)])(1, 1); -- { serverError ILLEGAL_AGGREGATION } DROP TABLE datetime; + +SET optimize_syntax_fuse_functions = 1; +SELECT quantile(1 AS a), quantile(a AS b), quantile(b AS c); From 7597763d6d260a36f082026fc0ba50b1d2c26f2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 06:55:00 +0300 Subject: [PATCH 382/950] Improve search for ar and ranlib --- cmake/linux/toolchain-aarch64.cmake | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index 94e81d13c2e..d414c2da823 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -10,8 +10,11 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-aarch set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") -set (CMAKE_AR "llvm-ar" CACHE FILEPATH "" FORCE) -set (CMAKE_RANLIB "llvm-ranlib" CACHE FILEPATH "" FORCE) +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") From 8196f89481ba14b08d02debca3daf5eec6c56514 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 06:58:08 +0300 Subject: [PATCH 383/950] Remove trash from CMake --- base/glibc-compatibility/memcpy/memcpy.h | 2 +- cmake/linux/toolchain-x86_64.cmake | 29 ++++ cmake/tools.cmake | 1 - contrib/CMakeLists.txt | 3 +- contrib/arrow-cmake/CMakeLists.txt | 11 +- contrib/arrow-cmake/orc_check.cmake | 138 ++---------------- .../grpc-cmake/protobuf_generate_grpc.cmake | 4 +- contrib/krb5-cmake/CMakeLists.txt | 2 +- .../CMake/CMakeTestCompileNestedException.cpp | 10 -- .../CMake/CMakeTestCompileSteadyClock.cpp | 7 - ...trerror.cpp => CMakeTestCompileStrerror.c} | 0 contrib/libhdfs3-cmake/CMake/Options.cmake | 25 +--- .../protobuf-cmake/protobuf_generate.cmake | 4 +- contrib/rocksdb-cmake/CMakeLists.txt | 34 +---- contrib/snappy-cmake/CMakeLists.txt | 42 ++++++ 15 files changed, 111 insertions(+), 201 deletions(-) create mode 100644 cmake/linux/toolchain-x86_64.cmake delete mode 100644 contrib/libhdfs3-cmake/CMake/CMakeTestCompileNestedException.cpp delete mode 100644 contrib/libhdfs3-cmake/CMake/CMakeTestCompileSteadyClock.cpp rename contrib/libhdfs3-cmake/CMake/{CMakeTestCompileStrerror.cpp => CMakeTestCompileStrerror.c} (100%) create mode 100644 contrib/snappy-cmake/CMakeLists.txt diff --git a/base/glibc-compatibility/memcpy/memcpy.h b/base/glibc-compatibility/memcpy/memcpy.h index 211d144cecb..9bee26a3722 100644 --- a/base/glibc-compatibility/memcpy/memcpy.h +++ b/base/glibc-compatibility/memcpy/memcpy.h @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake new file mode 100644 index 00000000000..6773bdbd64d --- /dev/null +++ b/cmake/linux/toolchain-x86_64.cmake @@ -0,0 +1,29 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + +set (CMAKE_SYSTEM_NAME "Linux") +set (CMAKE_SYSTEM_PROCESSOR "x86_64") +set (CMAKE_C_COMPILER_TARGET "x86_64-linux-gnu") +set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-gnu") +set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-gnu") + +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64") + +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-gnu/libc") + +set (CMAKE_AR "llvm-ar" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "llvm-ranlib" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index f94f4b289a3..4b2db0dedab 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -89,4 +89,3 @@ if (LINKER_NAME) message(STATUS "Using custom linker by name: ${LINKER_NAME}") endif () - diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 140cc0846ec..4d817c4c6e2 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -163,7 +163,7 @@ endif () if(USE_INTERNAL_SNAPPY_LIBRARY) set(SNAPPY_BUILD_TESTS 0 CACHE INTERNAL "") - add_subdirectory(snappy) + add_subdirectory(snappy-cmake) set (SNAPPY_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/snappy") endif() @@ -215,6 +215,7 @@ function(add_llvm) # Do not adjust RPATH in llvm, since then it will not be able to find libcxx/libcxxabi/libunwind set (CMAKE_INSTALL_RPATH "ON") + set (LLVM_COMPILER_CHECKED 1 CACHE INTERNAL "") set (LLVM_ENABLE_EH 1 CACHE INTERNAL "") set (LLVM_ENABLE_RTTI 1 CACHE INTERNAL "") set (LLVM_ENABLE_PIC 0 CACHE INTERNAL "") diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 427379dc9b2..841c280d192 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -54,7 +54,7 @@ target_link_libraries (${THRIFT_LIBRARY} PRIVATE boost::headers_only) set(ORC_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/orc/c++") set(ORC_INCLUDE_DIR "${ORC_SOURCE_DIR}/include") set(ORC_SOURCE_SRC_DIR "${ORC_SOURCE_DIR}/src") -set(ORC_SOURCE_WRAP_DIR "${ORC_SOURCE_DIR}/wrap") +# set(ORC_SOURCE_WRAP_DIR "${ORC_SOURCE_DIR}/wrap") set(ORC_BUILD_SRC_DIR "${CMAKE_CURRENT_BINARY_DIR}/../orc/c++/src") set(ORC_BUILD_INCLUDE_DIR "${CMAKE_CURRENT_BINARY_DIR}/../orc/c++/include") @@ -101,7 +101,14 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "AppleClang") set(CXX11_FLAGS "-std=c++0x") endif () -include("${ClickHouse_SOURCE_DIR}/contrib/orc/cmake_modules/CheckSourceCompiles.cmake") +set (ORC_CXX_HAS_INITIALIZER_LIST 1) +set (ORC_CXX_HAS_NOEXCEPT 1) +set (ORC_CXX_HAS_NULLPTR 1) +set (ORC_CXX_HAS_OVERRIDE 1) +set (ORC_CXX_HAS_UNIQUE_PTR 1) +set (ORC_CXX_HAS_CSTDINT 1) +set (ORC_CXX_HAS_THREAD_LOCAL 1) + include(orc_check.cmake) configure_file("${ORC_INCLUDE_DIR}/orc/orc-config.hh.in" "${ORC_BUILD_INCLUDE_DIR}/orc/orc-config.hh") configure_file("${ORC_SOURCE_SRC_DIR}/Adaptor.hh.in" "${ORC_BUILD_INCLUDE_DIR}/Adaptor.hh") diff --git a/contrib/arrow-cmake/orc_check.cmake b/contrib/arrow-cmake/orc_check.cmake index ad3b72e44cf..523e1cf1d86 100644 --- a/contrib/arrow-cmake/orc_check.cmake +++ b/contrib/arrow-cmake/orc_check.cmake @@ -1,130 +1,14 @@ -# Not changed part of contrib/orc/c++/src/CMakeLists.txt +set (HAS_PREAD 1) +set (HAS_STRPTIME 1) +set (HAS_STOLL 1) +set (INT64_IS_LL 1) +set (HAS_DIAGNOSTIC_PUSH 1) +set (HAS_STD_ISNAN 1) +set (HAS_STD_MUTEX 1) +set (NEEDS_REDUNDANT_MOVE 1) +set (HAS_PRE_1970 1) +set (HAS_POST_2038 1) +set (NEEDS_Z_PREFIX 0) -INCLUDE(CheckCXXSourceCompiles) - -CHECK_CXX_SOURCE_COMPILES(" - #include - #include - int main(int,char*[]){ - int f = open(\"/x/y\", O_RDONLY); - char buf[100]; - return pread(f, buf, 100, 1000) == 0; - }" - HAS_PREAD -) - -CHECK_CXX_SOURCE_COMPILES(" - #include - int main(int,char*[]){ - struct tm time2020; - return !strptime(\"2020-02-02 12:34:56\", \"%Y-%m-%d %H:%M:%S\", &time2020); - }" - HAS_STRPTIME -) - -CHECK_CXX_SOURCE_COMPILES(" - #include - int main(int,char* argv[]){ - return static_cast(std::stoll(argv[0])); - }" - HAS_STOLL -) - -CHECK_CXX_SOURCE_COMPILES(" - #include - #include - int main(int,char*[]){ - int64_t x = 1; printf(\"%lld\",x); - }" - INT64_IS_LL -) - -CHECK_CXX_SOURCE_COMPILES(" - #ifdef __clang__ - #pragma clang diagnostic push - #pragma clang diagnostic ignored \"-Wdeprecated\" - #pragma clang diagnostic pop - #elif defined(__GNUC__) - #pragma GCC diagnostic push - #pragma GCC diagnostic ignored \"-Wdeprecated\" - #pragma GCC diagnostic pop - #elif defined(_MSC_VER) - #pragma warning( push ) - #pragma warning( disable : 4996 ) - #pragma warning( pop ) - #else - unknownCompiler! - #endif - int main(int, char *[]) {}" - HAS_DIAGNOSTIC_PUSH -) - -CHECK_CXX_SOURCE_COMPILES(" - #include - int main(int, char *[]) { - return std::isnan(1.0f); - }" - HAS_STD_ISNAN -) - -CHECK_CXX_SOURCE_COMPILES(" - #include - int main(int, char *[]) { - std::mutex test_mutex; - std::lock_guard lock_mutex(test_mutex); - }" - HAS_STD_MUTEX -) - -CHECK_CXX_SOURCE_COMPILES(" - #include - std::string func() { - std::string var = \"test\"; - return std::move(var); - } - int main(int, char *[]) {}" - NEEDS_REDUNDANT_MOVE -) - -INCLUDE(CheckCXXSourceRuns) - -CHECK_CXX_SOURCE_RUNS(" - #include - int main(int, char *[]) { - time_t t = -14210715; // 1969-07-20 12:34:45 - struct tm *ptm = gmtime(&t); - return !(ptm && ptm->tm_year == 69); - }" - HAS_PRE_1970 -) - -CHECK_CXX_SOURCE_RUNS(" - #include - #include - int main(int, char *[]) { - setenv(\"TZ\", \"America/Los_Angeles\", 1); - tzset(); - struct tm time2037; - struct tm time2038; - strptime(\"2037-05-05 12:34:56\", \"%Y-%m-%d %H:%M:%S\", &time2037); - strptime(\"2038-05-05 12:34:56\", \"%Y-%m-%d %H:%M:%S\", &time2038); - return mktime(&time2038) - mktime(&time2037) != 31536000; - }" - HAS_POST_2038 -) - -set(CMAKE_REQUIRED_INCLUDES ${ZLIB_INCLUDE_DIR}) -set(CMAKE_REQUIRED_LIBRARIES zlib) -CHECK_CXX_SOURCE_COMPILES(" - #define Z_PREFIX - #include - z_stream strm; - int main(int, char *[]) { - deflateReset(&strm); - }" - NEEDS_Z_PREFIX -) - -# See https://cmake.org/cmake/help/v3.14/policy/CMP0075.html. Without unsetting it breaks thrift. set(CMAKE_REQUIRED_INCLUDES) set(CMAKE_REQUIRED_LIBRARIES) diff --git a/contrib/grpc-cmake/protobuf_generate_grpc.cmake b/contrib/grpc-cmake/protobuf_generate_grpc.cmake index 08d2976c26a..726428a7597 100644 --- a/contrib/grpc-cmake/protobuf_generate_grpc.cmake +++ b/contrib/grpc-cmake/protobuf_generate_grpc.cmake @@ -187,7 +187,7 @@ function(protobuf_generate_grpc) add_custom_command( OUTPUT ${_generated_srcs} - COMMAND protobuf::protoc + COMMAND $ ARGS --${protobuf_generate_grpc_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} --grpc_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} --plugin=protoc-gen-grpc=$ @@ -204,4 +204,4 @@ function(protobuf_generate_grpc) if(protobuf_generate_grpc_TARGET) target_sources(${protobuf_generate_grpc_TARGET} PRIVATE ${_generated_srcs_all}) endif() -endfunction() \ No newline at end of file +endfunction() diff --git a/contrib/krb5-cmake/CMakeLists.txt b/contrib/krb5-cmake/CMakeLists.txt index 7c750ca12b6..d6c3c23b14e 100644 --- a/contrib/krb5-cmake/CMakeLists.txt +++ b/contrib/krb5-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ find_program(AWK_PROGRAM awk) if(NOT AWK_PROGRAM) - message(FATAL_ERROR "You need the awk program to build ClickHouse with krb5 enabled.") + message(FATAL_ERROR "You need the awk program to build ClickHouse with krb5 enabled.") endif() set(KRB5_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/krb5/src") diff --git a/contrib/libhdfs3-cmake/CMake/CMakeTestCompileNestedException.cpp b/contrib/libhdfs3-cmake/CMake/CMakeTestCompileNestedException.cpp deleted file mode 100644 index 66918ca516e..00000000000 --- a/contrib/libhdfs3-cmake/CMake/CMakeTestCompileNestedException.cpp +++ /dev/null @@ -1,10 +0,0 @@ -#include -#include - -int main() { - try { - throw 2; - } catch (int) { - std::throw_with_nested(std::runtime_error("test")); - } -} diff --git a/contrib/libhdfs3-cmake/CMake/CMakeTestCompileSteadyClock.cpp b/contrib/libhdfs3-cmake/CMake/CMakeTestCompileSteadyClock.cpp deleted file mode 100644 index afcbe1b83b2..00000000000 --- a/contrib/libhdfs3-cmake/CMake/CMakeTestCompileSteadyClock.cpp +++ /dev/null @@ -1,7 +0,0 @@ -#include - -using std::chrono::steady_clock; - -void foo(const steady_clock &clock) { - return; -} diff --git a/contrib/libhdfs3-cmake/CMake/CMakeTestCompileStrerror.cpp b/contrib/libhdfs3-cmake/CMake/CMakeTestCompileStrerror.c similarity index 100% rename from contrib/libhdfs3-cmake/CMake/CMakeTestCompileStrerror.cpp rename to contrib/libhdfs3-cmake/CMake/CMakeTestCompileStrerror.c diff --git a/contrib/libhdfs3-cmake/CMake/Options.cmake b/contrib/libhdfs3-cmake/CMake/Options.cmake index 04ab823eedc..402aceac2fa 100644 --- a/contrib/libhdfs3-cmake/CMake/Options.cmake +++ b/contrib/libhdfs3-cmake/CMake/Options.cmake @@ -1,4 +1,4 @@ -OPTION(ENABLE_SSE "enable SSE4.2 buildin function" ON) +OPTION(ENABLE_SSE "enable SSE4.2 builtin function" ON) INCLUDE (CheckFunctionExists) CHECK_FUNCTION_EXISTS(dladdr HAVE_DLADDR) @@ -21,30 +21,21 @@ ADD_DEFINITIONS(-D_GNU_SOURCE) ADD_DEFINITIONS(-D_GLIBCXX_USE_NANOSLEEP) TRY_COMPILE(STRERROR_R_RETURN_INT - ${CMAKE_CURRENT_BINARY_DIR} - "${HDFS3_ROOT_DIR}/CMake/CMakeTestCompileStrerror.cpp" + ${CMAKE_CURRENT_BINARY_DIR} + "${CMAKE_CURRENT_SOURCE_DIR}/CMake/CMakeTestCompileStrerror.c" CMAKE_FLAGS "-DCMAKE_CXX_LINK_EXECUTABLE='echo not linking now...'" - OUTPUT_VARIABLE OUTPUT) + OUTPUT_VARIABLE OUTPUT) MESSAGE(STATUS "Checking whether strerror_r returns an int") IF(STRERROR_R_RETURN_INT) - MESSAGE(STATUS "Checking whether strerror_r returns an int -- yes") + MESSAGE(STATUS "Checking whether strerror_r returns an int -- yes") ELSE(STRERROR_R_RETURN_INT) - MESSAGE(STATUS "Checking whether strerror_r returns an int -- no") + MESSAGE(STATUS "Checking whether strerror_r returns an int -- no") ENDIF(STRERROR_R_RETURN_INT) -TRY_COMPILE(HAVE_STEADY_CLOCK - ${CMAKE_CURRENT_BINARY_DIR} - "${HDFS3_ROOT_DIR}/CMake/CMakeTestCompileSteadyClock.cpp" - CMAKE_FLAGS "-DCMAKE_CXX_LINK_EXECUTABLE='echo not linking now...'" - OUTPUT_VARIABLE OUTPUT) - -TRY_COMPILE(HAVE_NESTED_EXCEPTION - ${CMAKE_CURRENT_BINARY_DIR} - "${HDFS3_ROOT_DIR}/CMake/CMakeTestCompileNestedException.cpp" - CMAKE_FLAGS "-DCMAKE_CXX_LINK_EXECUTABLE='echo not linking now...'" - OUTPUT_VARIABLE OUTPUT) +set(HAVE_STEADY_CLOCK 1) +set(HAVE_NESTED_EXCEPTION 1) SET(HAVE_BOOST_CHRONO 0) SET(HAVE_BOOST_ATOMIC 0) diff --git a/contrib/protobuf-cmake/protobuf_generate.cmake b/contrib/protobuf-cmake/protobuf_generate.cmake index fc1dfd9cc11..c444162dd1e 100644 --- a/contrib/protobuf-cmake/protobuf_generate.cmake +++ b/contrib/protobuf-cmake/protobuf_generate.cmake @@ -181,11 +181,11 @@ function(protobuf_generate) add_custom_command( OUTPUT ${_generated_srcs} - COMMAND protobuf::protoc + COMMAND $ ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} DEPENDS ${_abs_file} protobuf::protoc COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" - VERBATIM ) + VERBATIM) endforeach() set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index e7ff1f548e3..6ccbd12516e 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -106,18 +106,6 @@ if(NOT MSVC) set(CMAKE_REQUIRED_FLAGS "-msse4.2 -mpclmul") endif() -CHECK_CXX_SOURCE_COMPILES(" -#include -#include -#include -int main() { - volatile uint32_t x = _mm_crc32_u32(0, 0); - const auto a = _mm_set_epi64x(0, 0); - const auto b = _mm_set_epi64x(0, 0); - const auto c = _mm_clmulepi64_si128(a, b, 0x00); - auto d = _mm_cvtsi128_si64(c); -} -" HAVE_SSE42) unset(CMAKE_REQUIRED_FLAGS) if(HAVE_SSE42) add_definitions(-DHAVE_SSE42) @@ -126,14 +114,7 @@ elseif(FORCE_SSE42) message(FATAL_ERROR "FORCE_SSE42=ON but unable to compile with SSE4.2 enabled") endif() -CHECK_CXX_SOURCE_COMPILES(" -#if defined(_MSC_VER) && !defined(__thread) -#define __thread __declspec(thread) -#endif -int main() { - static __thread int tls; -} -" HAVE_THREAD_LOCAL) +set (HAVE_THREAD_LOCAL 1) if(HAVE_THREAD_LOCAL) add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) endif() @@ -174,20 +155,13 @@ endif() option(WITH_FALLOCATE "build with fallocate" ON) if(WITH_FALLOCATE) - CHECK_CXX_SOURCE_COMPILES(" -#include -#include -int main() { - int fd = open(\"/dev/null\", 0); - fallocate(fd, FALLOC_FL_KEEP_SIZE, 0, 1024); -} -" HAVE_FALLOCATE) + set (HAVE_FALLOCATE 1) if(HAVE_FALLOCATE) add_definitions(-DROCKSDB_FALLOCATE_PRESENT) endif() endif() -CHECK_CXX_SOURCE_COMPILES(" +CHECK_C_SOURCE_COMPILES(" #include int main() { int fd = open(\"/dev/null\", 0); @@ -198,7 +172,7 @@ if(HAVE_SYNC_FILE_RANGE_WRITE) add_definitions(-DROCKSDB_RANGESYNC_PRESENT) endif() -CHECK_CXX_SOURCE_COMPILES(" +CHECK_C_SOURCE_COMPILES(" #include int main() { (void) PTHREAD_MUTEX_ADAPTIVE_NP; diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt new file mode 100644 index 00000000000..3f12d875d6b --- /dev/null +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -0,0 +1,42 @@ +set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/snappy") + +set(SNAPPY_IS_BIG_ENDIAN 0) + +include(CheckIncludeFile) +check_include_file("byteswap.h" HAVE_BYTESWAP_H) +check_include_file("sys/endian.h" HAVE_SYS_ENDIAN_H) +check_include_file("sys/mman.h" HAVE_SYS_MMAN_H) +check_include_file("sys/resource.h" HAVE_SYS_RESOURCE_H) +check_include_file("sys/time.h" HAVE_SYS_TIME_H) +check_include_file("sys/uio.h" HAVE_SYS_UIO_H) +check_include_file("unistd.h" HAVE_UNISTD_H) +check_include_file("windows.h" HAVE_WINDOWS_H) + +set (HAVE_BUILTIN_EXPECT 1) +set (HAVE_BUILTIN_CTZ 1) +set (SNAPPY_HAVE_SSSE3 HAVE_SSSE3) +set (HAVE_FUNC_MMAP 1) +set (HAVE_FUNC_SYSCONF 1) + +configure_file( + "${SOURCE_DIR}/cmake/config.h.in" + "${CMAKE_CURRENT_BINARY_DIR}/config.h") + +set(HAVE_SYS_UIO_H_01 1) + +configure_file( + "${SOURCE_DIR}/snappy-stubs-public.h.in" + "${CMAKE_CURRENT_BINARY_DIR}/snappy-stubs-public.h") + +add_library(snappy "") +target_sources(snappy + PRIVATE + "${SOURCE_DIR}/snappy-internal.h" + "${SOURCE_DIR}/snappy-stubs-internal.h" + "${SOURCE_DIR}/snappy-c.cc" + "${SOURCE_DIR}/snappy-sinksource.cc" + "${SOURCE_DIR}/snappy-stubs-internal.cc" + "${SOURCE_DIR}/snappy.cc") + +target_include_directories(snappy PUBLIC ${SOURCE_DIR} ${CMAKE_CURRENT_BINARY_DIR}) +target_compile_definitions(snappy PRIVATE -DHAVE_CONFIG_H) From 0dbe7c4036a03401b8ec89cfafc393bf9e6129ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:02:19 +0300 Subject: [PATCH 384/950] Remove old toolchain --- cmake/toolchain/linux-aarch64/README.txt | 2 -- docker/packager/binary/Dockerfile | 4 ---- docker/packager/binary/build.sh | 3 --- 3 files changed, 9 deletions(-) delete mode 100644 cmake/toolchain/linux-aarch64/README.txt diff --git a/cmake/toolchain/linux-aarch64/README.txt b/cmake/toolchain/linux-aarch64/README.txt deleted file mode 100644 index 3183d30b70d..00000000000 --- a/cmake/toolchain/linux-aarch64/README.txt +++ /dev/null @@ -1,2 +0,0 @@ -wget 'https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en' -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz --strip-components=1 diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 23012a38f9d..51d29b822e8 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -93,10 +93,6 @@ RUN git clone https://github.com/tpoechtrager/cctools-port.git \ # Download toolchain and SDK for Darwin RUN wget -nv https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacOSX11.0.sdk.tar.xz -# Download toolchain for ARM -# It contains all required headers and libraries. Note that it's named as "gcc" but actually we are using clang for cross compiling. -RUN wget -nv "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en" -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz - # Download toolchain for FreeBSD 11.3 RUN wget -nv https://clickhouse-datasets.s3.yandex.net/toolchains/toolchains/freebsd-11.3-toolchain.tar.xz diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 71402a2fd66..f78af924b66 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -6,9 +6,6 @@ mkdir -p build/cmake/toolchain/darwin-x86_64 tar xJf MacOSX11.0.sdk.tar.xz -C build/cmake/toolchain/darwin-x86_64 --strip-components=1 ln -sf darwin-x86_64 build/cmake/toolchain/darwin-aarch64 -mkdir -p build/cmake/toolchain/linux-aarch64 -tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C build/cmake/toolchain/linux-aarch64 --strip-components=1 - mkdir -p build/cmake/toolchain/freebsd-x86_64 tar xJf freebsd-11.3-toolchain.tar.xz -C build/cmake/toolchain/freebsd-x86_64 --strip-components=1 From 778793267b65500819156850768d970ef643a98f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:02:48 +0300 Subject: [PATCH 385/950] Add toolchain for linux-x86_64 --- cmake/linux/toolchain-x86_64.cmake | 7 +++++-- contrib/sysroot | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index 6773bdbd64d..5b23a4f6ebd 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -10,8 +10,11 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_6 set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-gnu/libc") -set (CMAKE_AR "llvm-ar" CACHE FILEPATH "" FORCE) -set (CMAKE_RANLIB "llvm-ranlib" CACHE FILEPATH "" FORCE) +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") diff --git a/contrib/sysroot b/contrib/sysroot index ae9209a6433..68bdc5b31a9 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit ae9209a643374c830a509ca75fd41d89ae189213 +Subproject commit 68bdc5b31a99a5291660bb9f6257071b75baf367 From 4d9604e29882743638a5cd2b5f373e887634aa20 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 11 Oct 2021 04:06:31 +0000 Subject: [PATCH 386/950] fix --- src/Common/ErrorCodes.cpp | 2 +- src/Storages/FileLog/DirectoryWatcherBase.cpp | 20 +++++++++++------ src/Storages/FileLog/DirectoryWatcherBase.h | 9 +++++--- .../FileLog/FileLogDirectoryWatcher.cpp | 6 +++-- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 22 +++++++++---------- 6 files changed, 36 insertions(+), 25 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a8c3ca62e30..6f33636d064 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,7 +589,7 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(624, NOT_REGULAR_FILE) \ - M(625, IO_ERROR) \ + M(625, IO_SETUP_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index ed263767ad2..350e71aadfd 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -13,11 +13,15 @@ namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; extern const int DIRECTORY_DOESNT_EXIST; - extern const int IO_ERROR; + extern const int IO_SETUP_ERROR; } -DirectoryWatcherBase::DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_) - : owner(owner_), path(path_), event_mask(event_mask_) +static constexpr int event_size = sizeof(struct inotify_event); +static constexpr int buffer_size = 1024 * (NAME_MAX + event_size + 1); + +DirectoryWatcherBase::DirectoryWatcherBase( + FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) + : WithContext(context_->getGlobalContext()), owner(owner_), path(path_), event_mask(event_mask_) { if (!std::filesystem::exists(path)) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist.", path); @@ -27,7 +31,8 @@ DirectoryWatcherBase::DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, con fd = inotify_init(); if (fd == -1) - throw Exception("cannot initialize inotify", ErrorCodes::IO_ERROR); + throw Exception("cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); } @@ -49,13 +54,13 @@ void DirectoryWatcherBase::watchFunc() int wd = inotify_add_watch(fd, path.c_str(), mask); if (wd == -1) { - owner.onError(Exception(ErrorCodes::IO_ERROR, "Watch directory {} failed.", path)); + owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed.", path)); } std::string buffer; - buffer.resize(4096); + buffer.resize(buffer_size); fd_set fds; - while (true) + while (!stopped) { FD_ZERO(&fds); FD_SET(fd, &fds); @@ -127,6 +132,7 @@ void DirectoryWatcherBase::start() void DirectoryWatcherBase::stop() { + stopped = true; if (watch_task) watch_task->deactivate(); } diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index 53fb3862a41..c9530453e2e 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -53,7 +53,7 @@ public: }; struct DirectoryEvent - { + { DirectoryEvent(const std::string & f, DirectoryEventType ev) : path(f), event(ev) { } /// The directory or file that has been changed. @@ -70,7 +70,8 @@ public: /// Creates a DirectoryWatcher for the directory given in path. /// To enable only specific events, an eventMask can be specified by /// OR-ing the desired event IDs (e.g., DW_ITEM_ADDED | DW_ITEM_MODIFIED). - explicit DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_ = DW_FILTER_ENABLE_ALL); + explicit DirectoryWatcherBase( + FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_ = DW_FILTER_ENABLE_ALL); ~DirectoryWatcherBase(); @@ -80,7 +81,7 @@ public: /// Returns the directory being watched. const std::string & directory() const; - [[noreturn]] void watchFunc(); + void watchFunc(); protected: void start(); @@ -92,6 +93,8 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; TaskThread watch_task; + std::atomic stopped{false}; + const std::string path; int event_mask; int fd; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index b0fbf5d535b..93aa3d0cf9f 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -2,8 +2,10 @@ namespace DB { -FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_unique(*this, path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) + : path(path_) + , dw(std::make_unique(*this, path, context_)) + , log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index c7216fa4872..b050d9a2c39 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -27,7 +27,7 @@ public: std::string error_msg = {}; }; - explicit FileLogDirectoryWatcher(const std::string & path_); + explicit FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_); ~FileLogDirectoryWatcher() = default; Events getEventsAndReset(); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 18515304c0c..00824c08690 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -86,7 +86,7 @@ StorageFileLog::StorageFileLog( #endif if (path_is_directory) - directory_watch = std::make_unique(path); + directory_watch = std::make_unique(path, context_); auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); @@ -791,20 +791,20 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & [event_path, event_infos] : events) + for (const auto & [file_name, event_infos] : events) { - String file_name = std::filesystem::path(event_path).filename(); + String file_path = getFullDataPath(file_name); for (const auto & event_info : event_infos) { switch (event_info.type) { case DirectoryWatcherBase::DW_ITEM_ADDED: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); /// Check if it is a regular file, and new file may be renamed or removed - if (std::filesystem::is_regular_file(event_path)) + if (std::filesystem::is_regular_file(file_path)) { - auto inode = getInode(event_path); + auto inode = getInode(file_path); file_infos.file_names.push_back(file_name); @@ -823,7 +823,7 @@ bool StorageFileLog::updateFileInfos() case DirectoryWatcherBase::DW_ITEM_MODIFIED: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); /// When new file added and appended, it has two event: DW_ITEM_ADDED /// and DW_ITEM_MODIFIED, since the order of these two events in the /// sequence is uncentain, so we may can not find it in file_infos, just @@ -836,21 +836,21 @@ bool StorageFileLog::updateFileInfos() case DirectoryWatcherBase::DW_ITEM_REMOVED: case DirectoryWatcherBase::DW_ITEM_MOVED_FROM: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) it->second.status = FileStatus::REMOVED; break; } case DirectoryWatcherBase::DW_ITEM_MOVED_TO: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); /// Similar to DW_ITEM_ADDED, but if it removed from an old file /// should obtain old meta file and rename meta file - if (std::filesystem::is_regular_file(event_path)) + if (std::filesystem::is_regular_file(file_path)) { file_infos.file_names.push_back(file_name); - auto inode = getInode(event_path); + auto inode = getInode(file_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) it->second = FileContext{.inode = inode}; From 1df4792a453b9adb680984611b4a74cd97af35a6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:07:58 +0300 Subject: [PATCH 387/950] Fix error --- cmake/linux/toolchain-aarch64.cmake | 2 +- cmake/linux/toolchain-x86_64.cmake | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/linux/toolchain-aarch64.cmake b/cmake/linux/toolchain-aarch64.cmake index d414c2da823..fa814d8d59b 100644 --- a/cmake/linux/toolchain-aarch64.cmake +++ b/cmake/linux/toolchain-aarch64.cmake @@ -11,7 +11,7 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-aarch set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/aarch64-linux-gnu/libc") find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") -find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index 5b23a4f6ebd..1e139cec062 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -11,7 +11,7 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_6 set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-gnu/libc") find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") -find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) From e5857618d8c6901e7d3b8a5e7865241ce05e11c6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 11 Oct 2021 04:12:04 +0000 Subject: [PATCH 388/950] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 237b0184424..39fd359765a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 237b018442435f732911e9fc6dd1d4d5328968a5 +Subproject commit 39fd359765a3a77b46d94ec3c5def3c7802a920f From d8bd1474bc2db7ded8c0c7ac560c0a51f36c6520 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 11 Oct 2021 04:18:21 +0000 Subject: [PATCH 389/950] fix --- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 93aa3d0cf9f..bd4a7543366 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -5,7 +5,7 @@ namespace DB FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) : path(path_) , dw(std::make_unique(*this, path, context_)) - , log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) + , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) { } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 00824c08690..c8ffd44369e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -86,7 +86,7 @@ StorageFileLog::StorageFileLog( #endif if (path_is_directory) - directory_watch = std::make_unique(path, context_); + directory_watch = std::make_unique(root_data_path, context_); auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); From 1338f3daf8671248f85c44d6d96e097d674ed411 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:44:12 +0300 Subject: [PATCH 390/950] Add missing files to libc-headers. --- contrib/libc-headers | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libc-headers b/contrib/libc-headers index a720b7105a6..aa5429bf67a 160000 --- a/contrib/libc-headers +++ b/contrib/libc-headers @@ -1 +1 @@ -Subproject commit a720b7105a610acbd7427eea475a5b6810c151eb +Subproject commit aa5429bf67a346e48ad60efd88bcefc286644bf3 From c581a40a366fbd49c30aec6665fa5a970d02be1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:46:08 +0300 Subject: [PATCH 391/950] Update sysroot --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index 68bdc5b31a9..bb46f9d9237 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 68bdc5b31a99a5291660bb9f6257071b75baf367 +Subproject commit bb46f9d92379def88cb1376ee3852ee60913ef83 From 0ec1f3bd3fa8a963c0472f4baa7202677dde36d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:48:41 +0300 Subject: [PATCH 392/950] Fix error --- contrib/snappy-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index 3f12d875d6b..57f79e42452 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -14,7 +14,7 @@ check_include_file("windows.h" HAVE_WINDOWS_H) set (HAVE_BUILTIN_EXPECT 1) set (HAVE_BUILTIN_CTZ 1) -set (SNAPPY_HAVE_SSSE3 HAVE_SSSE3) +set (SNAPPY_HAVE_SSSE3 ENABLE_SSSE3) set (HAVE_FUNC_MMAP 1) set (HAVE_FUNC_SYSCONF 1) From 31e9214e430751d738cd5962e0e239a4f31cecf2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 07:52:05 +0300 Subject: [PATCH 393/950] Fix error --- contrib/rocksdb-cmake/CMakeLists.txt | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 6ccbd12516e..19cb0bf2aa6 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -155,7 +155,14 @@ endif() option(WITH_FALLOCATE "build with fallocate" ON) if(WITH_FALLOCATE) - set (HAVE_FALLOCATE 1) + CHECK_C_SOURCE_COMPILES(" +#include +#include +int main() { + int fd = open(\"/dev/null\", 0); + fallocate(fd, FALLOC_FL_KEEP_SIZE, 0, 1024); +} +" HAVE_FALLOCATE) if(HAVE_FALLOCATE) add_definitions(-DROCKSDB_FALLOCATE_PRESENT) endif() From 264b278e1ea2b66c31f4ede47bbe8f5d074aa47f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 08:29:50 +0300 Subject: [PATCH 394/950] Fix error --- contrib/snappy-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index 57f79e42452..efa93d1064a 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -14,7 +14,7 @@ check_include_file("windows.h" HAVE_WINDOWS_H) set (HAVE_BUILTIN_EXPECT 1) set (HAVE_BUILTIN_CTZ 1) -set (SNAPPY_HAVE_SSSE3 ENABLE_SSSE3) +set (SNAPPY_HAVE_SSSE3 ${ENABLE_SSSE3}) set (HAVE_FUNC_MMAP 1) set (HAVE_FUNC_SYSCONF 1) From e7d3ead754728e3d70c8ef05b7225dd50ce84284 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 08:52:41 +0300 Subject: [PATCH 395/950] Fix error --- contrib/snappy-cmake/CMakeLists.txt | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/snappy-cmake/CMakeLists.txt b/contrib/snappy-cmake/CMakeLists.txt index efa93d1064a..0407e8bb30d 100644 --- a/contrib/snappy-cmake/CMakeLists.txt +++ b/contrib/snappy-cmake/CMakeLists.txt @@ -14,10 +14,15 @@ check_include_file("windows.h" HAVE_WINDOWS_H) set (HAVE_BUILTIN_EXPECT 1) set (HAVE_BUILTIN_CTZ 1) -set (SNAPPY_HAVE_SSSE3 ${ENABLE_SSSE3}) set (HAVE_FUNC_MMAP 1) set (HAVE_FUNC_SYSCONF 1) +if (ARCH_AMD64 AND ENABLE_SSSE3) + set (SNAPPY_HAVE_SSSE3 1) +else () + set (SNAPPY_HAVE_SSSE3 0) +endif () + configure_file( "${SOURCE_DIR}/cmake/config.h.in" "${CMAKE_CURRENT_BINARY_DIR}/config.h") From 284e547bc3fae8dafe2bf7c021aade39673ab472 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 09:01:23 +0300 Subject: [PATCH 396/950] Fix error --- src/Interpreters/JIT/CHJIT.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/JIT/CHJIT.cpp b/src/Interpreters/JIT/CHJIT.cpp index 83b0430a07d..9eec82b4179 100644 --- a/src/Interpreters/JIT/CHJIT.cpp +++ b/src/Interpreters/JIT/CHJIT.cpp @@ -156,7 +156,7 @@ public: throwFromErrno("Cannot mprotect memory region", ErrorCodes::CANNOT_MPROTECT); llvm::sys::Memory::InvalidateInstructionCache(block.base(), block.blockSize()); - InvalidateCache = false; + invalidate_cache = false; } # endif int res = mprotect(block.base(), block.blockSize(), protection_flags); From 3386e3a34962645fd4f4626e00ddd7cadf750625 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:24:27 +0300 Subject: [PATCH 397/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index c91d5caef8d..b05c236feac 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2353,7 +2353,7 @@ SELECT dummy, shardNum(), shardCount() FROM shard_num_example; **См. также** -- [Distributed Table Engine](../../engines/table-engines/special/distributed.md) +- Табличный движок [Distributed](../../engines/table-engines/special/distributed.md) ## shardCount {#shard-count} From dc01e86fb9798b867598c51fbc287df162bee015 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:24:46 +0300 Subject: [PATCH 398/950] Update docs/ru/engines/table-engines/special/distributed.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/table-engines/special/distributed.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/special/distributed.md b/docs/ru/engines/table-engines/special/distributed.md index ff1dc7c4057..3d7b8cf32d3 100644 --- a/docs/ru/engines/table-engines/special/distributed.md +++ b/docs/ru/engines/table-engines/special/distributed.md @@ -143,7 +143,7 @@ logs - имя кластера в конфигурационном файле с !!! note "Примечание" Так как табличные функции [remote](../../../sql-reference/table-functions/remote.md) и [cluster](../../../sql-reference/table-functions/cluster.md) создают временную таблицу на движке `Distributed`, то в ней также доступен столбец `_shard_num`. -**Смотрите также** +**См. также** - общее описание [виртуальных столбцов](../../../engines/table-engines/index.md#table_engines-virtual_columns) - настройка [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) From b6cb640572f870642d1386465c0ecbe6297ca2ad Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:25:03 +0300 Subject: [PATCH 399/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index b05c236feac..029e53237a8 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2358,7 +2358,7 @@ SELECT dummy, shardNum(), shardCount() FROM shard_num_example; ## shardCount {#shard-count} Возвращает общее количество шардов для распределенного запроса. -Если запрос не распределенный, то возвращается константное значение `0`. +Если запрос не распределенный, то возвращается значение `0`. **Синтаксис** From d530a2d17b8e9742596fe9cd62795fcd0dc51a9c Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:25:19 +0300 Subject: [PATCH 400/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 029e53237a8..ca73ee75f50 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -2316,7 +2316,7 @@ SELECT count(DISTINCT t) FROM (SELECT initialQueryID() AS t FROM remote('127.0.0 ## shardNum {#shard-num} Возвращает индекс шарда, который обрабатывает часть данных распределенного запроса. Индексы начинаются с `1`. -Если запрос не распределенный, то возвращается константное значение `0`. +Если запрос не распределенный, то возвращается значение `0`. **Синтаксис** From dccfd8d2623007a7a2edc9455c260376e506a7c3 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:26:24 +0300 Subject: [PATCH 401/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index ca73ee75f50..001f704c219 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -653,8 +653,8 @@ SELECT ## buildId() {#buildid} -Возвращает ID сборки, сгенерированный компилятором для запущенного сервера ClickHouse. -Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. +Возвращает ID сборки, сгенерированный компилятором для данного сервера ClickHouse. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. ## rowNumberInBlock {#function-rownumberinblock} From 32a4c9b69cd9ee516cb2e796045af94790831cf1 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:43:10 +0300 Subject: [PATCH 402/950] Update docs/ru/sql-reference/functions/date-time-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index d4777faf354..924dd559cbe 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -26,7 +26,7 @@ SELECT ## timeZone {#timezone} Возвращает часовой пояс сервера. -Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. **Синтаксис** From 294b8c01f8134592d12ff33e6a63650c6c1718b8 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:43:23 +0300 Subject: [PATCH 403/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 001f704c219..3ad9192d9a1 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -8,7 +8,7 @@ toc_title: "Прочие функции" ## hostName() {#hostname} Возвращает строку - имя хоста, на котором эта функция была выполнена. При распределённой обработке запроса, это будет имя хоста удалённого сервера, если функция выполняется на удалённом сервере. -Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. ## getMacro {#getmacro} From 0d357f34842ef9cb5a2c6aadd4b83d2a022a9ee0 Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:43:34 +0300 Subject: [PATCH 404/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 3ad9192d9a1..7134a19864e 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -649,7 +649,7 @@ SELECT ## version() {#version} Возвращает версию сервера в виде строки. -Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. ## buildId() {#buildid} From e9bb7dd08ce9c45d1a430f323acf8ba38dbee3be Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 11 Oct 2021 09:43:51 +0300 Subject: [PATCH 405/950] Update docs/ru/sql-reference/functions/other-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 7134a19864e..925aac56968 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -644,7 +644,7 @@ SELECT ## uptime() {#uptime} Возвращает аптайм сервера в секундах. -Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями актуальными для каждого шарда. Иначе возвращается константа. +Если функция вызывается в контексте распределенной таблицы, то она генерирует обычный столбец со значениями, актуальными для каждого шарда. Иначе возвращается константа. ## version() {#version} From 063f9cffabf0365a21787aea7c70b8f8da3397c8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 11 Oct 2021 10:03:35 +0300 Subject: [PATCH 406/950] Allow memory profiler under sanitizers Only query profiler cannot work reliably with sanitizers (due to unwinding from signal handler), but memory profiler should be fine. Plus sometimes the problem appears only on build with sanitizers, so it will be useful to have memory related profiling in trace_log. Also there is a flaky check for stateless tests, that uses build with ASan, and now trace_log there is empty, which sometimes does not allow to debug further. --- programs/server/Server.cpp | 15 ++------------- src/Common/QueryProfiler.cpp | 23 +++++++++++++++-------- src/Core/Defines.h | 15 +++++++++++++++ src/Core/Settings.h | 4 ++-- src/Core/SettingsQuirks.cpp | 28 ++++++++++++++++++++++++++++ 5 files changed, 62 insertions(+), 23 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index bfa402a6c21..902865c6de5 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1139,18 +1139,7 @@ if (ThreadFuzzer::instance().isEffective()) /// Init trace collector only after trace_log system table was created /// Disable it if we collect test coverage information, because it will work extremely slow. - /// - /// It also cannot work with sanitizers. - /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) - /// And they do unwinding frequently (on every malloc/free, thread/mutex operations, etc). - /// They change %rbp during unwinding and it confuses libunwind if signal comes during sanitizer unwinding - /// and query profiler decide to unwind stack with libunwind at this moment. - /// - /// Symptoms: you'll get silent Segmentation Fault - without sanitizer message and without usual ClickHouse diagnostics. - /// - /// Look at compiler-rt/lib/sanitizer_common/sanitizer_stacktrace.h - /// -#if USE_UNWIND && !WITH_COVERAGE && !defined(SANITIZER) && defined(__x86_64__) +#if USE_UNWIND && !WITH_COVERAGE && defined(__x86_64__) /// Profilers cannot work reliably with any other libunwind or without PHDR cache. if (hasPHDRCache()) { @@ -1182,7 +1171,7 @@ if (ThreadFuzzer::instance().isEffective()) #endif #if defined(SANITIZER) - LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they cannot work under sanitizers" + LOG_INFO(log, "Query Profiler disabled because they cannot work under sanitizers" " when two different stack unwinding methods will interfere with each other."); #endif diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 983ae864575..7b905937e11 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -82,7 +82,21 @@ QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const : log(&Poco::Logger::get("QueryProfiler")) , pause_signal(pause_signal_) { -#if USE_UNWIND +#if defined(SANITIZER) + UNUSED(thread_id); + UNUSED(clock_type); + UNUSED(period); + UNUSED(pause_signal); + + throw Exception("QueryProfiler disabled because they cannot work under sanitizers", ErrorCodes::NOT_IMPLEMENTED); +#elif !USE_UNWIND + UNUSED(thread_id); + UNUSED(clock_type); + UNUSED(period); + UNUSED(pause_signal); + + throw Exception("QueryProfiler cannot work with stock libunwind", ErrorCodes::NOT_IMPLEMENTED); +#else /// Sanity check. if (!hasPHDRCache()) throw Exception("QueryProfiler cannot be used without PHDR cache, that is not available for TSan build", ErrorCodes::NOT_IMPLEMENTED); @@ -144,13 +158,6 @@ QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const tryCleanup(); throw; } -#else - UNUSED(thread_id); - UNUSED(clock_type); - UNUSED(period); - UNUSED(pause_signal); - - throw Exception("QueryProfiler cannot work with stock libunwind", ErrorCodes::NOT_IMPLEMENTED); #endif } diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 660ae187627..215bf6780d9 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -64,3 +64,18 @@ /// Max depth of hierarchical dictionary #define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000 + +/// Query profiler cannot work with sanitizers. +/// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) +/// And they do unwinding frequently (on every malloc/free, thread/mutex operations, etc). +/// They change %rbp during unwinding and it confuses libunwind if signal comes during sanitizer unwinding +/// and query profiler decide to unwind stack with libunwind at this moment. +/// +/// Symptoms: you'll get silent Segmentation Fault - without sanitizer message and without usual ClickHouse diagnostics. +/// +/// Look at compiler-rt/lib/sanitizer_common/sanitizer_stacktrace.h +#if !defined(SANITIZER) +#define QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS 1000000000 +#else +#define QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS 0 +#endif diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f55f10c0267..6dcbf663dd5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -265,8 +265,8 @@ class IColumn; M(Bool, allow_suspicious_codecs, false, "If it is set to true, allow to specify meaningless compression codecs.", 0) \ M(Bool, allow_experimental_codecs, false, "If it is set to true, allow to specify experimental compression codecs (but we don't have those yet and this option does nothing).", 0) \ M(UInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \ - M(UInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ - M(UInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ + M(UInt64, query_profiler_real_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ + M(UInt64, query_profiler_cpu_time_period_ns, QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ M(Bool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \ M(String, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \ M(Float, opentelemetry_start_trace_probability, 0., "Probability to start an OpenTelemetry trace for an incoming query.", 0) \ diff --git a/src/Core/SettingsQuirks.cpp b/src/Core/SettingsQuirks.cpp index 94bd2e166b2..9b5644b75e1 100644 --- a/src/Core/SettingsQuirks.cpp +++ b/src/Core/SettingsQuirks.cpp @@ -6,6 +6,9 @@ #include #include +namespace +{ + /// Detect does epoll_wait with nested epoll fds works correctly. /// Polling nested epoll fds from epoll_wait is required for async_socket_for_remote and use_hedged_requests. /// @@ -31,6 +34,15 @@ bool nestedEpollWorks(Poco::Logger * log) return true; } +/// See also QUERY_PROFILER_DEFAULT_SAMPLE_RATE_NS in Core/Defines.h +#if !defined(SANITIZER) +bool queryProfilerWorks() { return true; } +#else +bool queryProfilerWorks() { return false; } +#endif + +} + namespace DB { @@ -52,6 +64,22 @@ void applySettingsQuirks(Settings & settings, Poco::Logger * log) LOG_WARNING(log, "use_hedged_requests has been disabled (you can explicitly enable it still)"); } } + + if (!queryProfilerWorks()) + { + if (settings.query_profiler_real_time_period_ns) + { + settings.query_profiler_real_time_period_ns = 0; + if (log) + LOG_WARNING(log, "query_profiler_real_time_period_ns has been disabled (due to server had been compiled with sanitizers)"); + } + if (settings.query_profiler_cpu_time_period_ns) + { + settings.query_profiler_cpu_time_period_ns = 0; + if (log) + LOG_WARNING(log, "query_profiler_cpu_time_period_ns has been disabled (due to server had been compiled with sanitizers)"); + } + } } } From ebfb013ea1577aa601a0caad45af60e12ba851b8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 8 Oct 2021 11:40:29 +0800 Subject: [PATCH 407/950] Fix potential leak of query_id_holder --- src/Processors/QueryPlan/QueryIdHolder.cpp | 1 + src/Processors/QueryPlan/QueryIdHolder.h | 5 +- .../QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 19 ++++-- src/Storages/MergeTree/MergeTreeData.h | 7 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 63 ++++++++++--------- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- .../01666_merge_tree_max_query_limit.sh | 8 +++ 8 files changed, 66 insertions(+), 41 deletions(-) diff --git a/src/Processors/QueryPlan/QueryIdHolder.cpp b/src/Processors/QueryPlan/QueryIdHolder.cpp index 87f6f892cd1..6ff238e017c 100644 --- a/src/Processors/QueryPlan/QueryIdHolder.cpp +++ b/src/Processors/QueryPlan/QueryIdHolder.cpp @@ -3,6 +3,7 @@ namespace DB { + QueryIdHolder::QueryIdHolder(const String & query_id_, const MergeTreeData & data_) : query_id(query_id_), data(data_) { } diff --git a/src/Processors/QueryPlan/QueryIdHolder.h b/src/Processors/QueryPlan/QueryIdHolder.h index ed8f9ec1d6b..1e1ee1af0a1 100644 --- a/src/Processors/QueryPlan/QueryIdHolder.h +++ b/src/Processors/QueryPlan/QueryIdHolder.h @@ -2,13 +2,16 @@ #include +#include + namespace DB { + class MergeTreeData; /// Holds the current query id and do something meaningful in destructor. /// Currently it's used for cleaning query id in the MergeTreeData query set. -struct QueryIdHolder +struct QueryIdHolder : private boost::noncopyable { QueryIdHolder(const std::string & query_id_, const MergeTreeData & data_); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index a48adc2d645..8d3005e725f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -945,7 +945,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons ProfileEvents::increment(ProfileEvents::SelectedRanges, result.selected_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, result.selected_marks); - auto query_id_holder = MergeTreeDataSelectExecutor::checkLimits(data, result.parts_with_ranges, context); + auto query_id_holder = MergeTreeDataSelectExecutor::checkLimits(data, result, context); if (result.parts_with_ranges.empty()) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c04e0d2e38f..646737b11ae 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5321,26 +5321,33 @@ void MergeTreeData::setDataVolume(size_t bytes, size_t rows, size_t parts) total_active_size_parts.store(parts, std::memory_order_release); } -void MergeTreeData::insertQueryIdOrThrow(const String & query_id, size_t max_queries) const +bool MergeTreeData::insertQueryIdOrThrow(const String & query_id, size_t max_queries) const { std::lock_guard lock(query_id_set_mutex); + return insertQueryIdOrThrowNoLock(query_id, max_queries, lock); +} + +bool MergeTreeData::insertQueryIdOrThrowNoLock(const String & query_id, size_t max_queries, const std::lock_guard &) const +{ if (query_id_set.find(query_id) != query_id_set.end()) - return; + return false; if (query_id_set.size() >= max_queries) throw Exception( ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for table {}. Maximum is: {}", log_name, max_queries); query_id_set.insert(query_id); + return true; } void MergeTreeData::removeQueryId(const String & query_id) const { std::lock_guard lock(query_id_set_mutex); + removeQueryIdNoLock(query_id, lock); +} + +void MergeTreeData::removeQueryIdNoLock(const String & query_id, const std::lock_guard &) const +{ if (query_id_set.find(query_id) == query_id_set.end()) - { - /// Do not throw exception, because this method is used in destructor. LOG_WARNING(log, "We have query_id removed but it's not recorded. This is a bug"); - assert(false); - } else query_id_set.erase(query_id); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bdebd5e9187..e33e6aa0ef4 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -794,11 +794,16 @@ public: /// section from config.xml. CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; + std::lock_guard getQueryIdSetLock() const { return std::lock_guard(query_id_set_mutex); } + /// Record current query id where querying the table. Throw if there are already `max_queries` queries accessing the same table. - void insertQueryIdOrThrow(const String & query_id, size_t max_queries) const; + /// Returns false if the `query_id` already exists in the running set, otherwise return true. + bool insertQueryIdOrThrow(const String & query_id, size_t max_queries) const; + bool insertQueryIdOrThrowNoLock(const String & query_id, size_t max_queries, const std::lock_guard &) const; /// Remove current query id after query finished. void removeQueryId(const String & query_id) const; + void removeQueryIdNoLock(const String & query_id, const std::lock_guard &) const; /// Return the partition expression types as a Tuple type. Return DataTypeUInt8 if partition expression is empty. DataTypePtr getPartitionValueType() const; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 03d76a7f79b..44b913ea81a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -993,47 +993,48 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd std::shared_ptr MergeTreeDataSelectExecutor::checkLimits( const MergeTreeData & data, - const RangesInDataParts & parts_with_ranges, + const ReadFromMergeTree::AnalysisResult & result, const ContextPtr & context) { const auto & settings = context->getSettingsRef(); - // Check limitations. query_id is used as the quota RAII's resource key. - String query_id; + const auto data_settings = data.getSettings(); + auto max_partitions_to_read + = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; + if (max_partitions_to_read > 0) { - const auto data_settings = data.getSettings(); - auto max_partitions_to_read - = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; - if (max_partitions_to_read > 0) - { - std::set partitions; - for (const auto & part_with_ranges : parts_with_ranges) - partitions.insert(part_with_ranges.data_part->info.partition_id); - if (partitions.size() > size_t(max_partitions_to_read)) - throw Exception( - ErrorCodes::TOO_MANY_PARTITIONS, - "Too many partitions to read. Current {}, max {}", - partitions.size(), - max_partitions_to_read); - } + std::set partitions; + for (const auto & part_with_ranges : result.parts_with_ranges) + partitions.insert(part_with_ranges.data_part->info.partition_id); + if (partitions.size() > size_t(max_partitions_to_read)) + throw Exception( + ErrorCodes::TOO_MANY_PARTITIONS, + "Too many partitions to read. Current {}, max {}", + partitions.size(), + max_partitions_to_read); + } - if (data_settings->max_concurrent_queries > 0 && data_settings->min_marks_to_honor_max_concurrent_queries > 0) + if (data_settings->max_concurrent_queries > 0 && data_settings->min_marks_to_honor_max_concurrent_queries > 0 + && result.selected_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) + { + auto query_id = context->getCurrentQueryId(); + if (!query_id.empty()) { - size_t sum_marks = 0; - for (const auto & part : parts_with_ranges) - sum_marks += part.getMarksCount(); - - if (sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) + auto lock = data.getQueryIdSetLock(); + if (data.insertQueryIdOrThrowNoLock(query_id, data_settings->max_concurrent_queries, lock)) { - query_id = context->getCurrentQueryId(); - if (!query_id.empty()) - data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries); + try + { + return std::make_shared(query_id, data); + } + catch (...) + { + /// If we fail to construct the holder, remove query_id explicitly to avoid leak. + data.removeQueryIdNoLock(query_id, lock); + throw; + } } } } - - if (!query_id.empty()) - return std::make_shared(query_id, data); - return nullptr; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 92c4382dc90..3cc5033c9f1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -197,7 +197,7 @@ public: /// Also, return QueryIdHolder. If not null, we should keep it until query finishes. static std::shared_ptr checkLimits( const MergeTreeData & data, - const RangesInDataParts & parts_with_ranges, + const ReadFromMergeTree::AnalysisResult & result, const ContextPtr & context); }; diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index c5fbb35a9cd..6fb337f2ca5 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -66,6 +66,14 @@ echo "yes" ${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '$query_id' SYNC FORMAT Null" wait +# Check correctness of multiple subqueries +query_id=max_concurrent_queries_$RANDOM +${CLICKHOUSE_CLIENT} --query_id "$query_id" --query "select i from simple where j in (select i from simple where i < 10)" + +# We have to grep the server's error log because the following warning message +# is generated during pipeline destruction and thus is not sent to the client. +grep -E -q "{$query_id} .*We have query_id removed but it's not recorded. This is a bug" /var/log/clickhouse-server/clickhouse-server.err.log && exit 1 + ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " drop table simple " From 2c55807b2a07e7c657f4c6eaa41d5edbb50b86c3 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Fri, 8 Oct 2021 19:56:25 +0300 Subject: [PATCH 408/950] Create dictionary with comment --- src/Dictionaries/DictionaryFactory.cpp | 6 +++++- src/Dictionaries/IDictionary.h | 13 +++++++++++++ .../getDictionaryConfigurationFromAST.cpp | 9 +++++++++ src/Storages/System/StorageSystemDictionaries.cpp | 13 ++++++++++++- .../02024_create_dictionary_with_comment.reference | 2 ++ .../02024_create_dictionary_with_comment.sql | 3 ++- 6 files changed, 43 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/DictionaryFactory.cpp b/src/Dictionaries/DictionaryFactory.cpp index 9b7fbbf7c68..92ff269a5ac 100644 --- a/src/Dictionaries/DictionaryFactory.cpp +++ b/src/Dictionaries/DictionaryFactory.cpp @@ -55,7 +55,11 @@ DictionaryPtr DictionaryFactory::create( if (found != registered_layouts.end()) { const auto & layout_creator = found->second.layout_create_function; - return layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); + auto result = layout_creator(name, dict_struct, config, config_prefix, std::move(source_ptr), global_context, created_from_ddl); + if (config.hasProperty(config_prefix + ".comment")) + result->setDictionaryComment(config.getString(config_prefix + ".comment")); + + return result; } } diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index a62fb4a788e..b40dc418c66 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -216,12 +216,25 @@ struct IDictionary : public IExternalLoadable return std::static_pointer_cast(IExternalLoadable::shared_from_this()); } + void setDictionaryComment(String new_comment) + { + std::lock_guard lock{name_mutex}; + dictionary_comment = std::move(new_comment); + } + + String getDictionaryComment() const + { + std::lock_guard lock{name_mutex}; + return dictionary_comment; + } + private: mutable std::mutex name_mutex; mutable StorageID dictionary_id; protected: const String full_name; + String dictionary_comment; }; } diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 341a89bfc06..0c6944eeccc 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -573,6 +573,15 @@ getDictionaryConfigurationFromAST(const ASTCreateQuery & query, ContextPtr conte if (query.dictionary->range) buildRangeConfiguration(xml_document, structure_element, query.dictionary->range, all_attr_names_and_types); + if (query.comment) + { + AutoPtr comment_element(xml_document->createElement("comment")); + current_dictionary->appendChild(comment_element); + AutoPtr comment_value(xml_document->createTextNode(query.comment->as()->value.safeGet())); + + comment_element->appendChild(comment_value); + } + conf->load(xml_document); return conf; } diff --git a/src/Storages/System/StorageSystemDictionaries.cpp b/src/Storages/System/StorageSystemDictionaries.cpp index 0826bb58473..d8f92d38081 100644 --- a/src/Storages/System/StorageSystemDictionaries.cpp +++ b/src/Storages/System/StorageSystemDictionaries.cpp @@ -47,7 +47,8 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() {"last_successful_update_time", std::make_shared()}, {"loading_duration", std::make_shared()}, //{ "creation_time", std::make_shared() }, - {"last_exception", std::make_shared()} + {"last_exception", std::make_shared()}, + {"comment", std::make_shared()} }; } @@ -140,6 +141,16 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, ContextPt else res_columns[i++]->insertDefault(); + if (dict_ptr) + res_columns[i++]->insert(dict_ptr->getDictionaryComment()); + else + { + if (load_result.config && load_result.config->config->has("dictionary.comment")) + res_columns[i++]->insert(load_result.config->config->getString("dictionary.comment")); + else + res_columns[i++]->insertDefault(); + } + /// Start fill virtual columns res_columns[i++]->insert(dictionary_structure.getKeyDescription()); diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference b/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference index e69de29bb2d..45f2c41f0b0 100644 --- a/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference +++ b/tests/queries/0_stateless/02024_create_dictionary_with_comment.reference @@ -0,0 +1,2 @@ +CREATE DICTIONARY default.`2024_dictionary_with_comment`\n(\n `id` UInt64,\n `value` String\n)\nPRIMARY KEY id\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT tcpPort() TABLE \'source_table\'))\nLIFETIME(MIN 0 MAX 1000)\nLAYOUT(FLAT())\nCOMMENT \'Test dictionary with comment\' +Test dictionary with comment diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql index de71d0d976b..bbe2fa7066b 100644 --- a/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql +++ b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql @@ -28,5 +28,6 @@ LIFETIME(MIN 0 MAX 1000) COMMENT 'Test dictionary with comment'; SHOW CREATE DICTIONARY 2024_dictionary_with_comment; +SELECT comment FROM system.dictionaries WHERE name == '2024_dictionary_with_comment' AND database == currentDatabase(); -SELECT comment FROM system.dictionaries WHERE name == '2024_dictionary_with_comment'; \ No newline at end of file +DROP DICTIONARY IF EXISTS 2024_dictionary_with_comment; \ No newline at end of file From e05d04153790b9ed5aac4382f1d83f428eb94d2e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 11 Oct 2021 11:34:58 +0300 Subject: [PATCH 409/950] Tests naming fix --- ...default.reference => 02026_accurate_cast_or_default.reference} | 0 ...ate_cast_or_default.sql => 02026_accurate_cast_or_default.sql} | 0 .../0_stateless/{2027_ngrams.reference => 02027_ngrams.reference} | 0 tests/queries/0_stateless/{2027_ngrams.sql => 02027_ngrams.sql} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{2026_accurate_cast_or_default.reference => 02026_accurate_cast_or_default.reference} (100%) rename tests/queries/0_stateless/{2026_accurate_cast_or_default.sql => 02026_accurate_cast_or_default.sql} (100%) rename tests/queries/0_stateless/{2027_ngrams.reference => 02027_ngrams.reference} (100%) rename tests/queries/0_stateless/{2027_ngrams.sql => 02027_ngrams.sql} (100%) diff --git a/tests/queries/0_stateless/2026_accurate_cast_or_default.reference b/tests/queries/0_stateless/02026_accurate_cast_or_default.reference similarity index 100% rename from tests/queries/0_stateless/2026_accurate_cast_or_default.reference rename to tests/queries/0_stateless/02026_accurate_cast_or_default.reference diff --git a/tests/queries/0_stateless/2026_accurate_cast_or_default.sql b/tests/queries/0_stateless/02026_accurate_cast_or_default.sql similarity index 100% rename from tests/queries/0_stateless/2026_accurate_cast_or_default.sql rename to tests/queries/0_stateless/02026_accurate_cast_or_default.sql diff --git a/tests/queries/0_stateless/2027_ngrams.reference b/tests/queries/0_stateless/02027_ngrams.reference similarity index 100% rename from tests/queries/0_stateless/2027_ngrams.reference rename to tests/queries/0_stateless/02027_ngrams.reference diff --git a/tests/queries/0_stateless/2027_ngrams.sql b/tests/queries/0_stateless/02027_ngrams.sql similarity index 100% rename from tests/queries/0_stateless/2027_ngrams.sql rename to tests/queries/0_stateless/02027_ngrams.sql From 2d069acc220347942ac3716168ded3dc7f9ded12 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 8 Oct 2021 16:13:56 +0300 Subject: [PATCH 410/950] System table data skipping indices added size --- .../system-tables/data_skipping_indices.md | 9 +++ docs/en/operations/system-tables/parts.md | 9 +++ src/Storages/IStorage.h | 7 ++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 50 +++++++++++++- src/Storages/MergeTree/IMergeTreeDataPart.h | 21 +++++- src/Storages/MergeTree/MergeTreeData.cpp | 68 ++++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 20 ++++-- .../MergeTree/MergedBlockOutputStream.cpp | 3 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- .../StorageSystemDataSkippingIndices.cpp | 19 ++++++ src/Storages/System/StorageSystemParts.cpp | 10 +++ 11 files changed, 192 insertions(+), 26 deletions(-) diff --git a/docs/en/operations/system-tables/data_skipping_indices.md b/docs/en/operations/system-tables/data_skipping_indices.md index 683666e1f77..add89ae9144 100644 --- a/docs/en/operations/system-tables/data_skipping_indices.md +++ b/docs/en/operations/system-tables/data_skipping_indices.md @@ -10,6 +10,9 @@ Columns: - `type` ([String](../../sql-reference/data-types/string.md)) — Index type. - `expr` ([String](../../sql-reference/data-types/string.md)) — Expression for the index calculation. - `granularity` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The number of granules in the block. +- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of compressed data, in bytes. +- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of decompressed data, in bytes. +- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — The size of marks, in bytes. **Example** @@ -26,6 +29,9 @@ name: clicks_idx type: minmax expr: clicks granularity: 1 +data_compressed_bytes: 58 +data_uncompressed_bytes: 6 +marks: 48 Row 2: ────── @@ -35,4 +41,7 @@ name: contacts_null_idx type: minmax expr: assumeNotNull(contacts_null) granularity: 1 +data_compressed_bytes: 58 +data_uncompressed_bytes: 6 +marks: 48 ``` diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 51a0a1180f3..45fdcc40451 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -38,6 +38,12 @@ Columns: - `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks. +- `secondary_indices_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of compressed data for secondary indices in the data part. All the auxiliary files (for example, files with marks) are not included. + +- `secondary_indices_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Total size of uncompressed data for secondary indices in the data part. All the auxiliary files (for example, files with marks) are not included. + +- `secondary_indices_marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The size of the file with marks for secondary indices. + - `modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – The time the directory with the data part was modified. This usually corresponds to the time of data part creation. - `remove_time` ([DateTime](../../sql-reference/data-types/datetime.md)) – The time when the data part became inactive. @@ -119,6 +125,9 @@ rows: 6 bytes_on_disk: 310 data_compressed_bytes: 157 data_uncompressed_bytes: 91 +secondary_indices_compressed_bytes: 58 +secondary_indices_uncompressed_bytes: 6 +secondary_indices_marks_bytes: 48 marks_bytes: 144 modification_time: 2020-06-18 13:01:49 remove_time: 1970-01-01 00:00:00 diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6ce17552ba1..0a9d1113601 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -87,6 +87,8 @@ struct ColumnSize } }; +using IndexSize = ColumnSize; + /** Storage. Describes the table. Responsible for * - storage of the table data; * - the definition in which files (or not in files) the data is stored; @@ -163,6 +165,11 @@ public: using ColumnSizeByName = std::unordered_map; virtual ColumnSizeByName getColumnSizes() const { return {}; } + /// Optional size information of each secondary index. + /// Valid only for MergeTree family. + using IndexSizeByName = std::unordered_map; + virtual IndexSizeByName getSecondaryIndexSizes() const { return {}; } + /// Get mutable version (snapshot) of storage metadata. Metadata object is /// multiversion, so it can be concurrently changed, but returned copy can be /// used without any locks. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index dc2c5f8185d..1a6290580a0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -584,7 +584,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadColumns(require_columns_checksums); loadChecksums(require_columns_checksums); loadIndexGranularity(); - calculateColumnsSizesOnDisk(); + calculateColumnsAndSecondaryIndicesSizesOnDisk(); loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity` loadRowsCount(); /// Must be called after loadIndexGranularity() as it uses the value of `index_granularity`. loadPartitionAndMinMaxIndex(); @@ -1420,6 +1420,11 @@ void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) cons throw Exception("Method 'checkConsistency' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); } +void IMergeTreeDataPart::calculateColumnsAndSecondaryIndicesSizesOnDisk() +{ + calculateColumnsSizesOnDisk(); + calculateSecondaryIndicesSizesOnDisk(); +} void IMergeTreeDataPart::calculateColumnsSizesOnDisk() { @@ -1429,6 +1434,40 @@ void IMergeTreeDataPart::calculateColumnsSizesOnDisk() calculateEachColumnSizes(columns_sizes, total_columns_size); } +void IMergeTreeDataPart::calculateSecondaryIndicesSizesOnDisk() +{ + if (checksums.empty()) + throw Exception("Cannot calculate secondary indexes sizes when columns or checksums are not initialized", ErrorCodes::LOGICAL_ERROR); + + auto secondary_indices_descriptions = storage.getInMemoryMetadataPtr()->secondary_indices; + + for (auto & index_description : secondary_indices_descriptions) + { + ColumnSize index_size; + + auto index_ptr = MergeTreeIndexFactory::instance().get(index_description); + auto index_name = index_ptr->getFileName(); + auto index_name_escaped = escapeForFileName(index_name); + + auto index_file_name = index_name_escaped + index_ptr->getSerializedFileExtension(); + auto index_marks_file_name = index_name_escaped + index_granularity_info.marks_file_extension; + + auto bin_checksum = checksums.files.find(index_file_name); + if (bin_checksum != checksums.files.end()) + { + index_size.data_compressed = bin_checksum->second.file_size; + index_size.data_uncompressed = bin_checksum->second.uncompressed_size; + } + + auto mrk_checksum = checksums.files.find(index_marks_file_name); + if (mrk_checksum != checksums.files.end()) + index_size.marks = mrk_checksum->second.file_size; + + total_secondary_indices_size.add(index_size); + secondary_index_sizes[index_description.name] = index_size; + } +} + ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & /* type */) const { /// For some types of parts columns_size maybe not calculated @@ -1439,6 +1478,15 @@ ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const I return ColumnSize{}; } +IndexSize IMergeTreeDataPart::getSecondaryIndexSize(const String & secondary_index_name) const +{ + auto it = secondary_index_sizes.find(secondary_index_name); + if (it != secondary_index_sizes.end()) + return it->second; + + return ColumnSize{}; +} + void IMergeTreeDataPart::accumulateColumnSizes(ColumnToSize & column_to_size) const { for (const auto & [column_name, size] : columns_sizes) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index be48aed5c8b..ceb3ed64170 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -55,6 +55,8 @@ public: using ColumnSizeByName = std::unordered_map; using NameToNumber = std::unordered_map; + using IndexSizeByName = std::unordered_map; + using Type = MergeTreeDataPartType; @@ -101,9 +103,16 @@ public: /// Otherwise return information about column size on disk. ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const; + /// NOTE: Returns zeros if secondary indexes are not found in checksums. + /// Otherwise return information about secondary index size on disk. + IndexSize getSecondaryIndexSize(const String & secondary_index_name) const; + /// Return information about column size on disk for all columns in part ColumnSize getTotalColumnsSize() const { return total_columns_size; } + /// Return information about secondary indexes size on disk for all indexes in part + IndexSize getTotalSeconaryIndicesSize() const { return total_secondary_indices_size; } + virtual String getFileNameForColumn(const NameAndTypePair & column) const = 0; virtual ~IMergeTreeDataPart(); @@ -341,7 +350,9 @@ public: /// Calculate the total size of the entire directory with all the files static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from); - void calculateColumnsSizesOnDisk(); + + /// Calculate column and secondary indices sizes on disk. + void calculateColumnsAndSecondaryIndicesSizesOnDisk(); String getRelativePathForPrefix(const String & prefix) const; @@ -396,6 +407,10 @@ protected: /// Size for each column, calculated once in calcuateColumnSizesOnDisk ColumnSizeByName columns_sizes; + ColumnSize total_secondary_indices_size; + + IndexSizeByName secondary_index_sizes; + /// Total size on disk, not only columns. May not contain size of /// checksums.txt and columns.txt. 0 - if not counted; UInt64 bytes_on_disk{0}; @@ -450,6 +465,10 @@ private: void loadPartitionAndMinMaxIndex(); + void calculateColumnsSizesOnDisk(); + + void calculateSecondaryIndicesSizesOnDisk(); + /// Load default compression codec from file default_compression_codec.txt /// if it not exists tries to deduce codec from compressed column without /// any specifial compression. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c04e0d2e38f..10e5fe9e71f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1167,7 +1167,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } - calculateColumnSizesImpl(); + calculateColumnAndSecondaryIndexSizesImpl(); LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); @@ -2352,7 +2352,7 @@ bool MergeTreeData::renameTempPartAndReplace( { covered_part->remove_time.store(current_time, std::memory_order_relaxed); modifyPartState(covered_part, DataPartState::Outdated); - removePartContributionToColumnSizes(covered_part); + removePartContributionToColumnAndSecondaryIndexSizes(covered_part); reduce_bytes += covered_part->getBytesOnDisk(); reduce_rows += covered_part->rows_count; ++reduce_parts; @@ -2361,7 +2361,7 @@ bool MergeTreeData::renameTempPartAndReplace( decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts); modifyPartState(part_it, DataPartState::Committed); - addPartContributionToColumnSizes(part); + addPartContributionToColumnAndSecondaryIndexSizes(part); addPartContributionToDataVolume(part); } @@ -2404,7 +2404,7 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect { if (part->getState() == IMergeTreeDataPart::State::Committed) { - removePartContributionToColumnSizes(part); + removePartContributionToColumnAndSecondaryIndexSizes(part); removePartContributionToDataVolume(part); } @@ -2542,7 +2542,7 @@ restore_covered) if (part->getState() == DataPartState::Committed) { removePartContributionToDataVolume(part); - removePartContributionToColumnSizes(part); + removePartContributionToColumnAndSecondaryIndexSizes(part); } modifyPartState(it_part, DataPartState::Deleting); @@ -2590,7 +2590,7 @@ restore_covered) if ((*it)->getState() != DataPartState::Committed) { - addPartContributionToColumnSizes(*it); + addPartContributionToColumnAndSecondaryIndexSizes(*it); addPartContributionToDataVolume(*it); modifyPartState(it, DataPartState::Committed); // iterator is not invalidated here } @@ -2621,7 +2621,7 @@ restore_covered) if ((*it)->getState() != DataPartState::Committed) { - addPartContributionToColumnSizes(*it); + addPartContributionToColumnAndSecondaryIndexSizes(*it); addPartContributionToDataVolume(*it); modifyPartState(it, DataPartState::Committed); } @@ -2973,32 +2973,46 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) part->modification_time = disk->getLastModified(full_part_path).epochTime(); } -void MergeTreeData::calculateColumnSizesImpl() +void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl() { + std::cerr << "MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl" << std::endl; + column_sizes.clear(); /// Take into account only committed parts auto committed_parts_range = getDataPartsStateRange(DataPartState::Committed); for (const auto & part : committed_parts_range) - addPartContributionToColumnSizes(part); + addPartContributionToColumnAndSecondaryIndexSizes(part); } -void MergeTreeData::addPartContributionToColumnSizes(const DataPartPtr & part) +void MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part) { + std::cerr << "MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes " << part->name << std::endl; + for (const auto & column : part->getColumns()) { + std::cerr << "Column name " << column.name << std::endl; ColumnSize & total_column_size = column_sizes[column.name]; + std::cerr << "Total column size compressed " << total_column_size.data_compressed << " uncompressed size " << total_column_size.data_uncompressed << std::endl; ColumnSize part_column_size = part->getColumnSize(column.name, *column.type); total_column_size.add(part_column_size); } + + auto indexes_descriptions = getInMemoryMetadataPtr()->secondary_indices; + for (const auto & index : indexes_descriptions) + { + IndexSize & total_secondary_index_size = secondary_index_sizes[index.name]; + IndexSize part_index_size = part->getSecondaryIndexSize(index.name); + total_secondary_index_size.add(part_index_size); + } } -void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part) +void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part) { for (const auto & column : part->getColumns()) { ColumnSize & total_column_size = column_sizes[column.name]; - ColumnSize part_column_size = part->getColumnSize(column.name, *column.type); + ColumnSize part_secondary_index_size = part->getColumnSize(column.name, *column.type); auto log_subtract = [&](size_t & from, size_t value, const char * field) { @@ -3009,9 +3023,29 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part from -= value; }; - log_subtract(total_column_size.data_compressed, part_column_size.data_compressed, ".data_compressed"); - log_subtract(total_column_size.data_uncompressed, part_column_size.data_uncompressed, ".data_uncompressed"); - log_subtract(total_column_size.marks, part_column_size.marks, ".marks"); + log_subtract(total_column_size.data_compressed, part_secondary_index_size.data_compressed, ".data_compressed"); + log_subtract(total_column_size.data_uncompressed, part_secondary_index_size.data_uncompressed, ".data_uncompressed"); + log_subtract(total_column_size.marks, part_secondary_index_size.marks, ".marks"); + } + + auto indexes_descriptions = getInMemoryMetadataPtr()->secondary_indices; + for (const auto & index : indexes_descriptions) + { + IndexSize & total_secondary_index_size = secondary_index_sizes[index.name]; + IndexSize part_secondary_index_size = part->getSecondaryIndexSize(index.name); + + auto log_subtract = [&](size_t & from, size_t value, const char * field) + { + if (value > from) + LOG_ERROR(log, "Possibly incorrect index size subtraction: {} - {} = {}, index: {}, field: {}", + from, value, from - value, index.name, field); + + from -= value; + }; + + log_subtract(total_secondary_index_size.data_compressed, part_secondary_index_size.data_compressed, ".data_compressed"); + log_subtract(total_secondary_index_size.data_uncompressed, part_secondary_index_size.data_uncompressed, ".data_uncompressed"); + log_subtract(total_secondary_index_size.marks, part_secondary_index_size.marks, ".marks"); } } @@ -4043,7 +4077,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: reduce_rows += covered_part->rows_count; data.modifyPartState(covered_part, DataPartState::Outdated); - data.removePartContributionToColumnSizes(covered_part); + data.removePartContributionToColumnAndSecondaryIndexSizes(covered_part); } reduce_parts += covered_parts.size(); @@ -4052,7 +4086,7 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: ++add_parts; data.modifyPartState(part, DataPartState::Committed); - data.addPartContributionToColumnSizes(part); + data.addPartContributionToColumnAndSecondaryIndexSizes(part); } } data.decreaseDataVolume(reduce_bytes, reduce_rows, reduce_parts); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bdebd5e9187..0e0e84d011b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -654,6 +654,12 @@ public: return column_sizes; } + IndexSizeByName getSecondaryIndexSizes() const override + { + auto lock = lockParts(); + return secondary_index_sizes; + } + /// For ATTACH/DETACH/DROP PARTITION. String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context) const; std::unordered_set getPartitionIDsFromQuery(const ASTs & asts, ContextPtr context) const; @@ -873,6 +879,9 @@ protected: /// Current column sizes in compressed and uncompressed form. ColumnSizeByName column_sizes; + /// Current secondary index sizes in compressed and uncompressed form. + IndexSizeByName secondary_index_sizes; + /// Engine-specific methods BrokenPartCallback broken_part_callback; @@ -1005,11 +1014,12 @@ protected: void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy) const; - /// Calculates column sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked. - void calculateColumnSizesImpl(); - /// Adds or subtracts the contribution of the part to compressed column sizes. - void addPartContributionToColumnSizes(const DataPartPtr & part); - void removePartContributionToColumnSizes(const DataPartPtr & part); + /// Calculates column and secondary indexes sizes in compressed form for the current state of data_parts. Call with data_parts mutex locked. + void calculateColumnAndSecondaryIndexSizesImpl(); + + /// Adds or subtracts the contribution of the part to compressed column and secondary indexes sizes. + void addPartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part); + void removePartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part); /// If there is no part in the partition with ID `partition_id`, returns empty ptr. Should be called under the lock. DataPartPtr getAnyPartInPartition(const String & partition_id, DataPartsLock & data_parts_lock) const; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 5206f77290b..43146709686 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -87,7 +87,8 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( new_part->checksums = checksums; new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->index_granularity = writer->getIndexGranularity(); - new_part->calculateColumnsSizesOnDisk(); + new_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(); + if (default_codec != nullptr) new_part->default_codec = default_codec; new_part->storage.lockSharedData(*new_part); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b8941fc9d84..115de043cd2 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -475,7 +475,7 @@ void finalizeMutatedPart( new_data_part->setBytesOnDisk( MergeTreeData::DataPart::calculateTotalSizeOnDisk(new_data_part->volume->getDisk(), new_data_part->getFullRelativePath())); new_data_part->default_codec = codec; - new_data_part->calculateColumnsSizesOnDisk(); + new_data_part->calculateColumnsAndSecondaryIndicesSizesOnDisk(); new_data_part->storage.lockSharedData(*new_data_part); } diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 7a6ce4ec519..d7fc06da953 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -25,6 +25,9 @@ StorageSystemDataSkippingIndices::StorageSystemDataSkippingIndices(const Storage { "type", std::make_shared() }, { "expr", std::make_shared() }, { "granularity", std::make_shared() }, + { "data_compressed_bytes", std::make_shared() }, + { "data_uncompressed_bytes", std::make_shared() }, + { "marks", std::make_shared()} })); setInMemoryMetadata(storage_metadata); } @@ -97,6 +100,7 @@ protected: continue; const auto indices = metadata_snapshot->getSecondaryIndices(); + auto secondary_index_sizes = table->getSecondaryIndexSizes(); for (const auto & index : indices) { ++rows_count; @@ -127,6 +131,21 @@ protected: // 'granularity' column if (column_mask[src_index++]) res_columns[res_index++]->insert(index.granularity); + + auto & secondary_index_size = secondary_index_sizes[index.name]; + + // 'compressed bytes' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(secondary_index_size.data_compressed); + + // 'uncompressed bytes' column + + if (column_mask[src_index++]) + res_columns[res_index++]->insert(secondary_index_size.data_uncompressed); + + /// 'marks' column + if (column_mask[src_index++]) + res_columns[res_index++]->insert(secondary_index_size.marks); } } } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index e79978463dd..6826082ef1d 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -30,6 +30,9 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"data_compressed_bytes", std::make_shared()}, {"data_uncompressed_bytes", std::make_shared()}, {"marks_bytes", std::make_shared()}, + {"secondary_indices_compressed_bytes", std::make_shared()}, + {"secondary_indices_uncompressed_bytes", std::make_shared()}, + {"secondary_indices_marks_bytes", std::make_shared()}, {"modification_time", std::make_shared()}, {"remove_time", std::make_shared()}, {"refcount", std::make_shared()}, @@ -98,6 +101,7 @@ void StorageSystemParts::processNextStorage( auto part_state = all_parts_state[part_number]; ColumnSize columns_size = part->getTotalColumnsSize(); + ColumnSize secondary_indexes_size = part->getTotalSeconaryIndicesSize(); size_t src_index = 0, res_index = 0; if (columns_mask[src_index++]) @@ -126,6 +130,12 @@ void StorageSystemParts::processNextStorage( columns[res_index++]->insert(columns_size.data_uncompressed); if (columns_mask[src_index++]) columns[res_index++]->insert(columns_size.marks); + if (columns_mask[src_index++]) + columns[res_index++]->insert(secondary_indexes_size.data_compressed); + if (columns_mask[src_index++]) + columns[res_index++]->insert(secondary_indexes_size.data_uncompressed); + if (columns_mask[src_index++]) + columns[res_index++]->insert(secondary_indexes_size.marks); if (columns_mask[src_index++]) columns[res_index++]->insert(static_cast(part->modification_time)); From ce0c41e1ad6a025a0117e7486ff9fd9a511c5be5 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 8 Oct 2021 23:43:16 +0300 Subject: [PATCH 411/950] Fixed tests --- src/Storages/MergeTree/MergeTreeData.cpp | 6 ------ ...1917_system_data_skipping_indices.reference | 10 +++++----- .../01932_alter_index_with_order.reference | 18 +++++++++--------- 3 files changed, 14 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 10e5fe9e71f..1ede7669832 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2975,8 +2975,6 @@ static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl() { - std::cerr << "MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl" << std::endl; - column_sizes.clear(); /// Take into account only committed parts @@ -2987,13 +2985,9 @@ void MergeTreeData::calculateColumnAndSecondaryIndexSizesImpl() void MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes(const DataPartPtr & part) { - std::cerr << "MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes " << part->name << std::endl; - for (const auto & column : part->getColumns()) { - std::cerr << "Column name " << column.name << std::endl; ColumnSize & total_column_size = column_sizes[column.name]; - std::cerr << "Total column size compressed " << total_column_size.data_compressed << " uncompressed size " << total_column_size.data_uncompressed << std::endl; ColumnSize part_column_size = part->getColumnSize(column.name, *column.type); total_column_size.add(part_column_size); } diff --git a/tests/queries/0_stateless/01917_system_data_skipping_indices.reference b/tests/queries/0_stateless/01917_system_data_skipping_indices.reference index b5a4b596a97..ca7e87e017b 100644 --- a/tests/queries/0_stateless/01917_system_data_skipping_indices.reference +++ b/tests/queries/0_stateless/01917_system_data_skipping_indices.reference @@ -1,8 +1,8 @@ -default data_01917 d1_idx minmax d1 1 -default data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1 -default data_01917_2 memory set frequency * length(name) 5 -default data_01917_2 sample_index1 minmax length(name), name 4 -default data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4 +test data_01917 d1_idx minmax d1 1 0 0 0 +test data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1 0 0 0 +test data_01917_2 memory set frequency * length(name) 5 0 0 0 +test data_01917_2 sample_index1 minmax length(name), name 4 0 0 0 +test data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4 0 0 0 2 3 d1_idx diff --git a/tests/queries/0_stateless/01932_alter_index_with_order.reference b/tests/queries/0_stateless/01932_alter_index_with_order.reference index 07e1aab3df9..eff9ea7da0e 100644 --- a/tests/queries/0_stateless/01932_alter_index_with_order.reference +++ b/tests/queries/0_stateless/01932_alter_index_with_order.reference @@ -1,9 +1,9 @@ -default alter_index_test index_a set a 1 -default alter_index_test index_b minmax b 1 -default alter_index_test index_c set c 2 -default alter_index_test index_a set a 1 -default alter_index_test index_d set d 1 -default alter_index_test index_b minmax b 1 -default alter_index_test index_c set c 2 -default alter_index_test index_a set a 1 -default alter_index_test index_d set d 1 +default alter_index_test index_a set a 1 0 0 0 +default alter_index_test index_b minmax b 1 0 0 0 +default alter_index_test index_c set c 2 0 0 0 +default alter_index_test index_a set a 1 0 0 0 +default alter_index_test index_d set d 1 0 0 0 +default alter_index_test index_b minmax b 1 0 0 0 +default alter_index_test index_c set c 2 0 0 0 +default alter_index_test index_a set a 1 0 0 0 +default alter_index_test index_d set d 1 0 0 0 From 61a725f53199697451200a2d24e0173347f8b9e2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 10 Oct 2021 23:53:31 +0300 Subject: [PATCH 412/950] Fixed tests --- .../01917_system_data_skipping_indices.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01917_system_data_skipping_indices.reference b/tests/queries/0_stateless/01917_system_data_skipping_indices.reference index ca7e87e017b..115d60f60cc 100644 --- a/tests/queries/0_stateless/01917_system_data_skipping_indices.reference +++ b/tests/queries/0_stateless/01917_system_data_skipping_indices.reference @@ -1,8 +1,8 @@ -test data_01917 d1_idx minmax d1 1 0 0 0 -test data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1 0 0 0 -test data_01917_2 memory set frequency * length(name) 5 0 0 0 -test data_01917_2 sample_index1 minmax length(name), name 4 0 0 0 -test data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4 0 0 0 +default data_01917 d1_idx minmax d1 1 0 0 0 +default data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1 0 0 0 +default data_01917_2 memory set frequency * length(name) 5 0 0 0 +default data_01917_2 sample_index1 minmax length(name), name 4 0 0 0 +default data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4 0 0 0 2 3 d1_idx From a07ce981214ab0f046afa23fb3f5551bc24a7270 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 11 Oct 2021 18:07:00 +0800 Subject: [PATCH 413/950] Use system.text_log in test --- .../01666_merge_tree_max_query_limit.sh | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index 6fb337f2ca5..e04c9515009 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -4,8 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function wait_for_query_to_start() -{ +function wait_for_query_to_start() { while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT sum(read_rows) FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done } @@ -21,14 +20,14 @@ insert into simple select number, number + 100 from numbers(5000); query_id="long_running_query-$CLICKHOUSE_DATABASE" echo "Spin up a long running query" -${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.1) from simple settings max_block_size = 1 format Null" --query_id "$query_id" > /dev/null 2>&1 & +${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.1) from simple settings max_block_size = 1 format Null" --query_id "$query_id" >/dev/null 2>&1 & wait_for_query_to_start "$query_id" # query which reads marks >= min_marks_to_honor_max_concurrent_queries is throttled echo "Check if another query with some marks to read is throttled" -${CLICKHOUSE_CLIENT} --query "select * from simple" 2> /dev/null; +${CLICKHOUSE_CLIENT} --query "select * from simple" 2>/dev/null CODE=$? -[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; +[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1 echo "yes" # query which reads marks less than min_marks_to_honor_max_concurrent_queries is allowed @@ -41,9 +40,9 @@ ${CLICKHOUSE_CLIENT} --query "alter table simple modify setting min_marks_to_hon # Now smaller queries are also throttled echo "Check if another query with less marks to read is throttled" -${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" 2> /dev/null; +${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" 2>/dev/null CODE=$? -[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; +[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1 echo "yes" echo "Modify max_concurrent_queries to 2" @@ -58,9 +57,9 @@ ${CLICKHOUSE_CLIENT} --query "alter table simple modify setting max_concurrent_q # Now queries are throttled again echo "Check if another query with less marks to read is throttled" -${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" 2> /dev/null; +${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" 2>/dev/null CODE=$? -[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; +[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1 echo "yes" ${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = '$query_id' SYNC FORMAT Null" @@ -70,10 +69,9 @@ wait query_id=max_concurrent_queries_$RANDOM ${CLICKHOUSE_CLIENT} --query_id "$query_id" --query "select i from simple where j in (select i from simple where i < 10)" -# We have to grep the server's error log because the following warning message +# We have to search the server's error log because the following warning message # is generated during pipeline destruction and thus is not sent to the client. -grep -E -q "{$query_id} .*We have query_id removed but it's not recorded. This is a bug" /var/log/clickhouse-server/clickhouse-server.err.log && exit 1 +${CLICKHOUSE_CLIENT} --query "system flush logs" +if [[ $(${CLICKHOUSE_CLIENT} --query "select count() > 0 from system.text_log where query_id = '$query_id' and level = 'Warning' and message like '%We have query_id removed but it\'s not recorded. This is a bug%' format TSVRaw") == 1 ]]; then echo "We have query_id removed but it's not recorded. This is a bug." >&2; exit 1; fi -${CLICKHOUSE_CLIENT} --multiline --multiquery --query " -drop table simple -" +${CLICKHOUSE_CLIENT} --query "drop table simple" From 9ad919d91a8da739b1aa8a3d708b608f34dbf583 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Oct 2021 13:25:54 +0300 Subject: [PATCH 414/950] More timeouts in stress test --- docker/test/stress/stress | 35 ++++++++++++++++++----------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 8fc4ade2da6..1559b084565 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -71,42 +71,42 @@ def prepare_for_hung_check(drop_databases): # FIXME this function should not exist, but... # ThreadFuzzer significantly slows down server and causes false-positive hung check failures - call("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'", shell=True, stderr=STDOUT) + call("clickhouse client -q 'SYSTEM STOP THREAD FUZZER'", shell=True, stderr=STDOUT, timeout=30) # We attach gdb to clickhouse-server before running tests # to print stacktraces of all crashes even if clickhouse cannot print it for some reason. # However, it obstruct checking for hung queries. logging.info("Will terminate gdb (if any)") - call("kill -TERM $(pidof gdb)", shell=True, stderr=STDOUT) + call("kill -TERM $(pidof gdb)", shell=True, stderr=STDOUT, timeout=30) # Some tests set too low memory limit for default user and forget to reset in back. # It may cause SYSTEM queries to fail, let's disable memory limit. - call("clickhouse client --max_memory_usage_for_user=0 -q 'SELECT 1 FORMAT Null'", shell=True, stderr=STDOUT) + call("clickhouse client --max_memory_usage_for_user=0 -q 'SELECT 1 FORMAT Null'", shell=True, stderr=STDOUT, timeout=30) # Some tests execute SYSTEM STOP MERGES or similar queries. # It may cause some ALTERs to hang. # Possibly we should fix tests and forbid to use such queries without specifying table. - call("clickhouse client -q 'SYSTEM START MERGES'", shell=True, stderr=STDOUT) - call("clickhouse client -q 'SYSTEM START DISTRIBUTED SENDS'", shell=True, stderr=STDOUT) - call("clickhouse client -q 'SYSTEM START TTL MERGES'", shell=True, stderr=STDOUT) - call("clickhouse client -q 'SYSTEM START MOVES'", shell=True, stderr=STDOUT) - call("clickhouse client -q 'SYSTEM START FETCHES'", shell=True, stderr=STDOUT) - call("clickhouse client -q 'SYSTEM START REPLICATED SENDS'", shell=True, stderr=STDOUT) - call("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'", shell=True, stderr=STDOUT) + call("clickhouse client -q 'SYSTEM START MERGES'", shell=True, stderr=STDOUT, timeout=30) + call("clickhouse client -q 'SYSTEM START DISTRIBUTED SENDS'", shell=True, stderr=STDOUT, timeout=30) + call("clickhouse client -q 'SYSTEM START TTL MERGES'", shell=True, stderr=STDOUT, timeout=30) + call("clickhouse client -q 'SYSTEM START MOVES'", shell=True, stderr=STDOUT, timeout=30) + call("clickhouse client -q 'SYSTEM START FETCHES'", shell=True, stderr=STDOUT, timeout=30) + call("clickhouse client -q 'SYSTEM START REPLICATED SENDS'", shell=True, stderr=STDOUT, timeout=30) + call("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'", shell=True, stderr=STDOUT, timeout=30) # Issue #21004, live views are experimental, so let's just suppress it - call("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """, shell=True, stderr=STDOUT) + call("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """, shell=True, stderr=STDOUT, timeout=30) # Kill other queries which known to be slow # It's query from 01232_preparing_sets_race_condition_long, it may take up to 1000 seconds in slow builds - call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """, shell=True, stderr=STDOUT) + call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """, shell=True, stderr=STDOUT, timeout=30) # Long query from 00084_external_agregation - call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT) + call("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """, shell=True, stderr=STDOUT, timeout=30) if drop_databases: # Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too. # Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds). - databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True).decode('utf-8').strip().split() + databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True, timeout=30).decode('utf-8').strip().split() for db in databases: if db == "system": continue @@ -117,13 +117,13 @@ def prepare_for_hung_check(drop_databases): # Wait for last queries to finish if any, not longer than 300 seconds call("""clickhouse client -q "select sleepEachRow(( select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300 - ) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT) + ) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT, timeout=30) # Even if all clickhouse-test processes are finished, there are probably some sh scripts, # which still run some new queries. Let's ignore them. try: query = """clickhouse client -q "SELECT count() FROM system.processes where where elapsed > 300" """ - output = check_output(query, shell=True, stderr=STDOUT).decode('utf-8').strip() + output = check_output(query, shell=True, stderr=STDOUT, timeout=30).decode('utf-8').strip() if int(output) == 0: return False except: @@ -176,6 +176,7 @@ if __name__ == "__main__": if res != 0 and have_long_running_queries: logging.info("Hung check failed with exit code {}".format(res)) hung_check_status = "Hung check failed\tFAIL\n" - open(os.path.join(args.output_folder, "test_results.tsv"), 'w+').write(hung_check_status) + with open(os.path.join(args.output_folder, "test_results.tsv"), 'w+') as results: + results.write(hung_check_status) logging.info("Stress test finished") From 83717b7c3b7e64ad752f53d2041bc8360e609715 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 29 Sep 2021 18:05:57 +0800 Subject: [PATCH 415/950] Get rid of naming limitation of projections. --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++++-- src/Storages/MergeTree/MergeTask.cpp | 16 +++++++++++----- src/Storages/MergeTree/MergeTask.h | 6 +++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- .../MergeTree/MergeTreeDataMergerMutator.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +--- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 8 ++++---- src/Storages/MergeTree/checkDataPart.cpp | 8 +++++--- src/Storages/ProjectionsDescription.cpp | 17 +++++++---------- 10 files changed, 41 insertions(+), 34 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index dc2c5f8185d..6ac53c68e84 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -439,9 +439,13 @@ void IMergeTreeDataPart::removeIfNeeded() if (file_name.empty()) throw Exception("relative_path " + relative_path + " of part " + name + " is invalid or not set", ErrorCodes::LOGICAL_ERROR); - if (!startsWith(file_name, "tmp")) + if (!startsWith(file_name, "tmp") && !endsWith(file_name, ".tmp_proj")) { - LOG_ERROR(storage.log, "~DataPart() should remove part {} but its name doesn't start with tmp. Too suspicious, keeping the part.", path); + LOG_ERROR( + storage.log, + "~DataPart() should remove part {} but its name doesn't start with \"tmp\" or end with \".tmp_proj\". Too " + "suspicious, keeping the part.", + path); return; } } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 357659b3bbb..c6e8dafd8b0 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -89,7 +89,10 @@ static void extractMergingAndGatheringColumns( bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() { - const String local_tmp_prefix = global_ctx->parent_part ? ctx->prefix : "tmp_merge_"; + // projection parts have different prefix and suffix compared to normal parts. + // E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge. + const String local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_"; + const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : ""; if (global_ctx->merges_blocker->isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); @@ -114,7 +117,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } ctx->disk = global_ctx->space_reservation->getDisk(); - auto local_new_part_tmp_path = global_ctx->data->relative_data_path + local_tmp_prefix + global_ctx->future_part->name + (global_ctx->parent_part ? ".proj" : "") + "/"; + auto local_new_part_relative_tmp_path = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix + "/"; + auto local_new_part_tmp_path = global_ctx->data->relative_data_path + local_new_part_relative_tmp_path; if (ctx->disk->exists(local_new_part_tmp_path)) throw Exception("Directory " + fullPath(ctx->disk, local_new_part_tmp_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); @@ -138,7 +142,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->future_part->type, global_ctx->future_part->part_info, local_single_disk_volume, - local_tmp_prefix + global_ctx->future_part->name + (global_ctx->parent_part ? ".proj" : ""), + local_new_part_relative_tmp_path, global_ctx->parent_part); global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; @@ -526,7 +530,9 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c auto projection_future_part = std::make_shared(); projection_future_part->assign(std::move(projection_parts)); projection_future_part->name = projection.name; - projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; + // TODO (ab): path in future_part is only for merge process introspection, which is not available for merges of projection parts. + // Let's comment this out to avoid code inconsistency and add it back after we implement projection merge introspection. + // projection_future_part->path = global_ctx->future_part->path + "/" + projection.name + ".proj/"; projection_future_part->part_info = {"all", 0, 0, 0}; MergeTreeData::MergingParams projection_merging_params; @@ -553,7 +559,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->deduplicate_by_columns, projection_merging_params, global_ctx->new_data_part.get(), - "", // empty string for projection + ".proj", global_ctx->data, global_ctx->merges_blocker, global_ctx->ttl_merges_blocker)); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 05903f94c91..22dc70bd78c 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -58,7 +58,7 @@ public: Names deduplicate_by_columns_, MergeTreeData::MergingParams merging_params_, const IMergeTreeDataPart * parent_part_, - String prefix_, + String suffix_, MergeTreeData * data_, ActionBlocker * merges_blocker_, ActionBlocker * ttl_merges_blocker_) @@ -83,7 +83,7 @@ public: auto prepare_stage_ctx = std::make_shared(); - prepare_stage_ctx->prefix = std::move(prefix_); + prepare_stage_ctx->suffix = std::move(suffix_); prepare_stage_ctx->merging_params = std::move(merging_params_); (*stages.begin())->setRuntimeContext(std::move(prepare_stage_ctx), global_ctx); @@ -170,7 +170,7 @@ private: struct ExecuteAndFinalizeHorizontalPartRuntimeContext : public IStageRuntimeContext //-V730 { /// Dependencies - String prefix; + String suffix; MergeTreeData::MergingParams merging_params{}; DiskPtr tmp_disk{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 903f4cd27fc..5d97c64b49b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -428,7 +428,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const Names & deduplicate_by_columns, const MergeTreeData::MergingParams & merging_params, const IMergeTreeDataPart * parent_part, - const String & prefix) + const String & suffix) { return std::make_shared( future_part, @@ -442,7 +442,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( deduplicate_by_columns, merging_params, parent_part, - prefix, + suffix, &data, &merges_blocker, &ttl_merges_blocker); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 9eb91d7fbf8..22650ac4eca 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -108,7 +108,7 @@ public: const Names & deduplicate_by_columns, const MergeTreeData::MergingParams & merging_params, const IMergeTreeDataPart * parent_part = nullptr, - const String & prefix = ""); + const String & suffix = ""); /// Mutate a single data part with the specified commands. Will create and return a temporary part. MutateTaskPtr mutatePartToTemporaryPart( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 03d76a7f79b..77a91af037e 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -173,9 +173,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( auto projection_plan = std::make_unique(); if (query_info.projection->desc->is_minmax_count_projection) { - Pipe pipe(std::make_shared( - query_info.minmax_count_projection_block.cloneEmpty(), - Chunk(query_info.minmax_count_projection_block.getColumns(), query_info.minmax_count_projection_block.rows()))); + Pipe pipe(std::make_shared(query_info.minmax_count_projection_block)); auto read_from_pipe = std::make_unique(std::move(pipe)); projection_plan->addStep(std::move(read_from_pipe)); } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d939312c0bb..752f85a1290 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -575,7 +575,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart( return writeProjectionPartImpl( part_name, part_type, - "tmp_insert_" + part_name + ".proj" /* relative_path */, + part_name + ".tmp_proj" /* relative_path */, true /* is_temp */, parent_part, data, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b8941fc9d84..fbc2f58d424 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -654,7 +654,7 @@ public: {}, projection_merging_params, ctx->new_data_part.get(), - "tmp_merge_"); + ".tmp_proj"); next_level_parts.push_back(executeHere(tmp_part_merge_task)); @@ -832,8 +832,8 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() auto projection_block = projection_squash.add({}); if (projection_block) { - projection_parts[projection.name].emplace_back( - MergeTreeDataWriter::writeTempProjectionPart(*ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num)); + projection_parts[projection.name].emplace_back(MergeTreeDataWriter::writeTempProjectionPart( + *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num)); } } @@ -1082,7 +1082,7 @@ private: if (!ctx->disk->isDirectory(it->path())) ctx->disk->createHardLink(it->path(), destination); - else if (!startsWith("tmp_", it->name())) // ignore projection tmp merge dir + else if (!endsWith(".tmp_proj", it->name())) // ignore projection tmp merge dir { // it's a projection part directory ctx->disk->createDirectories(destination); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8a234833da7..0af395fd1bd 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -102,7 +102,7 @@ IMergeTreeDataPart::Checksums checkDataPart( /// It also calculates checksum of projections. auto checksum_file = [&](const String & file_path, const String & file_name) { - if (disk->isDirectory(file_path) && endsWith(file_name, ".proj") && !startsWith(file_name, "tmp_")) // ignore projection tmp merge dir + if (disk->isDirectory(file_path) && endsWith(file_name, ".proj")) { auto projection_name = file_name.substr(0, file_name.size() - sizeof(".proj") + 1); auto pit = data_part->getProjectionParts().find(projection_name); @@ -124,7 +124,8 @@ IMergeTreeDataPart::Checksums checkDataPart( auto file_buf = disk->readFile(proj_path); HashingReadBuffer hashing_buf(*file_buf); hashing_buf.ignoreAll(); - projection_checksums_data.files[MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); + projection_checksums_data.files[MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION] + = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); } else { @@ -140,7 +141,8 @@ IMergeTreeDataPart::Checksums checkDataPart( [&](const ISerialization::SubstreamPath & substream_path) { String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin"; - checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name); + checksums_data.files[projection_file_name] + = checksum_compressed_file(disk, projection_path + projection_file_name); }, {}); } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index c0b96bd9f54..42294b8152c 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -89,9 +89,6 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const if (projection_definition->name.empty()) throw Exception("Projection must have name in definition.", ErrorCodes::INCORRECT_QUERY); - if (startsWith(projection_definition->name, "tmp_")) - throw Exception("Projection's name cannot start with 'tmp_'", ErrorCodes::INCORRECT_QUERY); - if (!projection_definition->query) throw Exception("QUERY is required for projection", ErrorCodes::INCORRECT_QUERY); @@ -220,13 +217,13 @@ void ProjectionDescription::recalculateWithNewColumns(const ColumnsDescription & Block ProjectionDescription::calculate(const Block & block, ContextPtr context) const { auto builder = InterpreterSelectQuery( - query_ast, - context, - Pipe(std::make_shared(block, Chunk(block.getColumns(), block.rows()))), - SelectQueryOptions{ - type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns - : QueryProcessingStage::WithMergeableState}) - .buildQueryPipeline(); + query_ast, + context, + Pipe(std::make_shared(block, Chunk(block.getColumns(), block.rows()))), + SelectQueryOptions{ + type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns + : QueryProcessingStage::WithMergeableState}) + .buildQueryPipeline(); builder.resize(1); builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); From b0d887a0fef89fb529cff4f7c02cfab8cf75c280 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 11 Oct 2021 14:00:10 +0300 Subject: [PATCH 416/950] Added tests --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 + ...28_system_data_skipping_indices_size.reference | 1 + .../2028_system_data_skipping_indices_size.sql | 15 +++++++++++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/2028_system_data_skipping_indices_size.reference create mode 100644 tests/queries/0_stateless/2028_system_data_skipping_indices_size.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1a6290580a0..0f701cc4adf 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1452,6 +1452,7 @@ void IMergeTreeDataPart::calculateSecondaryIndicesSizesOnDisk() auto index_file_name = index_name_escaped + index_ptr->getSerializedFileExtension(); auto index_marks_file_name = index_name_escaped + index_granularity_info.marks_file_extension; + /// If part does not contain index auto bin_checksum = checksums.files.find(index_file_name); if (bin_checksum != checksums.files.end()) { diff --git a/tests/queries/0_stateless/2028_system_data_skipping_indices_size.reference b/tests/queries/0_stateless/2028_system_data_skipping_indices_size.reference new file mode 100644 index 00000000000..d0378511850 --- /dev/null +++ b/tests/queries/0_stateless/2028_system_data_skipping_indices_size.reference @@ -0,0 +1 @@ +default test_table value_index minmax value 1 38 12 24 diff --git a/tests/queries/0_stateless/2028_system_data_skipping_indices_size.sql b/tests/queries/0_stateless/2028_system_data_skipping_indices_size.sql new file mode 100644 index 00000000000..e77f88aa36f --- /dev/null +++ b/tests/queries/0_stateless/2028_system_data_skipping_indices_size.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test_table; + +CREATE TABLE test_table +( + key UInt64, + value String, + INDEX value_index value TYPE minmax GRANULARITY 1 +) +Engine=MergeTree() +ORDER BY key; + +INSERT INTO test_table VALUES (0, 'Value'); +SELECT * FROM system.data_skipping_indices WHERE database = currentDatabase(); + +DROP TABLE test_table; From 95a69b9f4b6478f902c7a970296a8c5b2c193a9c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Oct 2021 14:05:52 +0300 Subject: [PATCH 417/950] Make test non endless --- .../01509_check_many_parallel_quorum_inserts_long.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index c2682cd8cfa..6533eeb12f5 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -18,9 +18,10 @@ for i in $(seq 1 $NUM_REPLICAS); do done function thread { - while true - do + i=0 retries=300 + while [[ $i -lt $retries ]]; do # server can be dead $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break + ((++i)) sleep 0.1 done } From 72bccaa50141cd1206d1d064bc2d767a68cb9f99 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 11 Oct 2021 19:12:08 +0800 Subject: [PATCH 418/950] Fix path name --- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/MergeTask.cpp | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index be48aed5c8b..b74b2ca3321 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -175,6 +175,7 @@ public: /// A directory path (relative to storage's path) where part data is actually stored /// Examples: 'detached/tmp_fetch_', 'tmp_', '' + /// NOTE: Cannot have trailing slash. mutable String relative_path; MergeTreeIndexGranularityInfo index_granularity_info; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index c6e8dafd8b0..aa3f91a4f00 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -117,8 +117,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } ctx->disk = global_ctx->space_reservation->getDisk(); - auto local_new_part_relative_tmp_path = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix + "/"; - auto local_new_part_tmp_path = global_ctx->data->relative_data_path + local_new_part_relative_tmp_path; + auto local_new_part_relative_tmp_path_name = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix; + auto local_new_part_tmp_path = global_ctx->data->relative_data_path + local_new_part_relative_tmp_path_name + "/"; if (ctx->disk->exists(local_new_part_tmp_path)) throw Exception("Directory " + fullPath(ctx->disk, local_new_part_tmp_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); @@ -142,7 +142,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->future_part->type, global_ctx->future_part->part_info, local_single_disk_volume, - local_new_part_relative_tmp_path, + local_new_part_relative_tmp_path_name, global_ctx->parent_part); global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; From 2370a8d6dabd9b7fdd2aaa4571b123053c60d835 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 14:28:46 +0300 Subject: [PATCH 419/950] Fix PVS-Studio --- src/Common/JSONBuilder.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Functions/FunctionsBinaryRepr.cpp | 4 ++-- src/Functions/FunctionsMiscellaneous.h | 2 +- .../ExternalUserDefinedExecutableFunctionsLoader.cpp | 10 +++++----- src/Storages/Kafka/KafkaSource.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 7 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Common/JSONBuilder.h b/src/Common/JSONBuilder.h index 12a2b129645..b1a1ec4ae6a 100644 --- a/src/Common/JSONBuilder.h +++ b/src/Common/JSONBuilder.h @@ -94,7 +94,7 @@ class JSONMap : public IItem }; public: - void add(std::string key, ItemPtr value) { values.emplace_back(Pair{.key = std::move(key), .value = std::move(value)}); } + void add(std::string key, ItemPtr value) { values.emplace_back(Pair{.key = std::move(key), .value = std::move(value)}); } //-V1030 void add(std::string key, std::string value) { add(std::move(key), std::make_unique(std::move(value))); } void add(std::string key, const char * value) { add(std::move(key), std::make_unique(value)); } void add(std::string key, std::string_view value) { add(std::move(key), std::make_unique(value)); } diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index d6a0c25e9f1..95e816ee45a 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -254,7 +254,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) .format =config.getString(settings_config_prefix + ".format", ""), .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), - .header_entries = std::move(header_entries) + .header_entries = std::move(header_entries) //-V1030 }; return std::make_unique(dict_struct, configuration, credentials, sample_block, context, created_from_ddl); diff --git a/src/Functions/FunctionsBinaryRepr.cpp b/src/Functions/FunctionsBinaryRepr.cpp index 7f0835d8edf..20b2acac88a 100644 --- a/src/Functions/FunctionsBinaryRepr.cpp +++ b/src/Functions/FunctionsBinaryRepr.cpp @@ -50,7 +50,7 @@ struct HexImpl UInt8 byte = x >> offset; /// Skip leading zeros - if (byte == 0 && !was_nonzero && offset) + if (byte == 0 && !was_nonzero && offset) //-V560 continue; was_nonzero = true; @@ -138,7 +138,7 @@ struct BinImpl UInt8 byte = x >> offset; /// Skip leading zeros - if (byte == 0 && !was_nonzero && offset) + if (byte == 0 && !was_nonzero && offset) //-V560 continue; was_nonzero = true; diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 32700cb692e..7e8cab842c8 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -238,7 +238,7 @@ public: capture = std::make_shared(Capture{ .captured_names = captured_names_, - .captured_types = std::move(captured_types), + .captured_types = std::move(captured_types), //-V1030 .lambda_arguments = lambda_arguments_, .return_name = expression_return_name_, .return_type = function_return_type_, diff --git a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp index f8d2c0a86ef..2de7b4b7846 100644 --- a/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp +++ b/src/Interpreters/ExternalUserDefinedExecutableFunctionsLoader.cpp @@ -107,11 +107,11 @@ ExternalLoader::LoadablePtr ExternalUserDefinedExecutableFunctionsLoader::create UserDefinedExecutableFunctionConfiguration function_configuration { .type = function_type, - .name = std::move(name), - .script_path = std::move(command), - .format = std::move(format), - .argument_types = std::move(argument_types), - .result_type = std::move(result_type), + .name = std::move(name), //-V1030 + .script_path = std::move(command), //-V1030 + .format = std::move(format), //-V1030 + .argument_types = std::move(argument_types), //-V1030 + .result_type = std::move(result_type), //-V1030 .pool_size = pool_size, .command_termination_timeout = command_termination_timeout, .max_command_execution_time = max_command_execution_time, diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 68fc17a97e5..ad48858b658 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -67,7 +67,7 @@ Chunk KafkaSource::generateImpl() broken = true; } - if (!buffer || is_finished) + if (is_finished) return {}; is_finished = true; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 03d76a7f79b..88b0878bb74 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -983,7 +983,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd index_stats.emplace_back(ReadFromMergeTree::IndexStat{ .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, - .description = std::move(description), + .description = std::move(description), //-V1030 .num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped, .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); } From e0a4fb31a9b37d78ef61c38816d9b38fc51ff48d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 14:34:05 +0300 Subject: [PATCH 420/950] Update submodules to simplify cross build --- contrib/grpc | 2 +- contrib/llvm | 2 +- contrib/protobuf | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/grpc b/contrib/grpc index 60c986e15ca..7eac189a6ba 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 60c986e15cae70aade721d26badabab1f822fdd6 +Subproject commit 7eac189a6badddac593580ec2ad1478bd2656fc7 diff --git a/contrib/llvm b/contrib/llvm index f30bbecef78..20607e61728 160000 --- a/contrib/llvm +++ b/contrib/llvm @@ -1 +1 @@ -Subproject commit f30bbecef78b75b527e257c1304d0be2f2f95975 +Subproject commit 20607e61728e97c969e536644c3c0c1bb1a50672 diff --git a/contrib/protobuf b/contrib/protobuf index 75601841d17..c1c5d020260 160000 --- a/contrib/protobuf +++ b/contrib/protobuf @@ -1 +1 @@ -Subproject commit 75601841d172c73ae6bf4ce8121f42b875cdbabd +Subproject commit c1c5d02026059f4c3cb51aaa08e82288d3e08b89 From e24be8dfb0461b02fb48fbab458f4c71d5e62a8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Oct 2021 14:41:49 +0300 Subject: [PATCH 421/950] Update submodule --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index bb46f9d9237..611d3315e9e 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit bb46f9d92379def88cb1376ee3852ee60913ef83 +Subproject commit 611d3315e9e369a338de4ffa128eb87b4fb87dec From 59a78830f9b73f9ff366449874047828019ad58e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Oct 2021 14:46:01 +0300 Subject: [PATCH 422/950] Better timeouts in clickhouse-test --- tests/clickhouse-test | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index f10e38b87e5..061333297e2 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -89,10 +89,13 @@ def make_clickhouse_client(base_args): # hence we should use 'system'. database='system', settings=get_additional_client_options_dict(base_args)) + def clickhouse_execute_one(base_args, *args, **kwargs): return make_clickhouse_client(base_args).execute_one(*args, **kwargs) + def clickhouse_execute(base_args, *args, **kwargs): return make_clickhouse_client(base_args).execute(*args, **kwargs) + def clickhouse_execute_pandas(base_args, *args, **kwargs): return make_clickhouse_client(base_args).execute_pandas(*args, **kwargs) @@ -109,6 +112,7 @@ def stop_tests(): global restarted_tests with stop_tests_triggered_lock: + print("Stopping tests") if not stop_tests_triggered.is_set(): stop_tests_triggered.set() @@ -875,7 +879,7 @@ def run_tests_array(all_tests_with_params): while True: if is_concurrent: - case = queue.get() + case = queue.get(timeout=args.timeout) if not case: break else: @@ -1076,10 +1080,10 @@ def do_run_tests(jobs, test_suite: TestSuite, parallel): pool.map_async(run_tests_array, parallel_tests_array) for suit in test_suite.parallel_tests: - queue.put(suit) + queue.put(suit, timeout=args.timeout) for _ in range(jobs): - queue.put(None) + queue.put(None, timeout=args.timeout) queue.close() From 0ca8660af60cc94697fb9b8a8d7c227d0c399cbd Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 11 Oct 2021 14:52:31 +0300 Subject: [PATCH 423/950] Update build-cross-arm.md --- docs/en/development/build-cross-arm.md | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/docs/en/development/build-cross-arm.md b/docs/en/development/build-cross-arm.md index 7c2b002d638..2b3f976783c 100644 --- a/docs/en/development/build-cross-arm.md +++ b/docs/en/development/build-cross-arm.md @@ -9,16 +9,9 @@ This is for the case when you have Linux machine and want to use it to build `cl The cross-build for AARCH64 is based on the [Build instructions](../development/build.md), follow them first. -## Install Clang-8 {#install-clang-8} +## Install Clang-13 Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup. -For example, in Ubuntu Bionic you can use the following commands: - -``` bash -echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" | sudo tee /etc/apt/sources.list.d/llvm.list -sudo apt-get update -sudo apt-get install clang-8 -``` ## Install Cross-Compilation Toolset {#install-cross-compilation-toolset} From 467b45f3b5de51233b6adac2de11222975bb9677 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 11 Oct 2021 14:52:44 +0300 Subject: [PATCH 424/950] Update build-cross-arm.md --- docs/en/development/build-cross-arm.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-cross-arm.md b/docs/en/development/build-cross-arm.md index 2b3f976783c..e4d0c170ff6 100644 --- a/docs/en/development/build-cross-arm.md +++ b/docs/en/development/build-cross-arm.md @@ -27,7 +27,7 @@ tar xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz -C build-aarch64/cma ``` bash cd ClickHouse mkdir build-arm64 -CC=clang-8 CXX=clang++-8 cmake . -Bbuild-arm64 -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-aarch64.cmake +CC=clang-13 CXX=clang++-13 cmake . -Bbuild-arm64 -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-aarch64.cmake ninja -C build-arm64 ``` From f2d97e322589bdc92df7c910068fb0ad08aa8ad9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 11 Oct 2021 14:53:12 +0300 Subject: [PATCH 425/950] Update build-cross-arm.md --- docs/en/development/build-cross-arm.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/development/build-cross-arm.md b/docs/en/development/build-cross-arm.md index e4d0c170ff6..eb99105a857 100644 --- a/docs/en/development/build-cross-arm.md +++ b/docs/en/development/build-cross-arm.md @@ -11,7 +11,10 @@ The cross-build for AARCH64 is based on the [Build instructions](../development/ ## Install Clang-13 -Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup. +Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup or do +``` +sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" +``` ## Install Cross-Compilation Toolset {#install-cross-compilation-toolset} From 351f2a3a842c6ddb63c09958c5e2397c1f42a093 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Mon, 11 Oct 2021 14:53:26 +0300 Subject: [PATCH 426/950] Update BorinSSL --- .gitmodules | 3 +- contrib/boringssl-cmake/CMakeLists.txt | 49 ++++++++++++++++++-------- 2 files changed, 35 insertions(+), 17 deletions(-) diff --git a/.gitmodules b/.gitmodules index d8c5aa640a8..74d1049ce01 100644 --- a/.gitmodules +++ b/.gitmodules @@ -212,8 +212,7 @@ url = https://github.com/ClickHouse-Extras/libpq [submodule "contrib/boringssl"] path = contrib/boringssl - url = https://github.com/FArthur-cmd/boringssl.git - branch = update_BoringSSL + url = https://github.com/ClickHouse-Extras/boringssl.git [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git diff --git a/contrib/boringssl-cmake/CMakeLists.txt b/contrib/boringssl-cmake/CMakeLists.txt index 4502d6e9d42..474e32f3b91 100644 --- a/contrib/boringssl-cmake/CMakeLists.txt +++ b/contrib/boringssl-cmake/CMakeLists.txt @@ -4,7 +4,7 @@ # This file is created by generate_build_files.py and edited accordingly. -cmake_minimum_required(VERSION 3.0) +cmake_minimum_required(VERSION 3.5) project(BoringSSL LANGUAGES C CXX) @@ -20,12 +20,7 @@ if(CMAKE_COMPILER_IS_GNUCXX OR CLANG) set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") endif() - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-common") - if((CMAKE_C_COMPILER_VERSION VERSION_GREATER "4.8.99") OR CLANG) - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -std=c11") - else() - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -std=c99") - endif() + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-common -std=c11") endif() # pthread_rwlock_t requires a feature flag. @@ -55,7 +50,7 @@ add_definitions(-DBORINGSSL_IMPLEMENTATION) # builds. if(NOT OPENSSL_NO_ASM AND CMAKE_OSX_ARCHITECTURES) list(LENGTH CMAKE_OSX_ARCHITECTURES NUM_ARCHES) - if(NOT ${NUM_ARCHES} EQUAL 1) + if(NOT NUM_ARCHES EQUAL 1) message(FATAL_ERROR "Universal binaries not supported.") endif() list(GET CMAKE_OSX_ARCHITECTURES 0 CMAKE_SYSTEM_PROCESSOR) @@ -78,7 +73,13 @@ elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "AMD64") elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "x86") set(ARCH "x86") elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "i386") - set(ARCH "x86") + # cmake uses `uname -p` to set the system processor, but Solaris + # systems support multiple architectures. + if((${CMAKE_SYSTEM_NAME} STREQUAL "SunOS") AND CMAKE_SIZEOF_VOID_P EQUAL 8) + set(ARCH "x86_64") + else() + set(ARCH "x86") + endif() elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "i686") set(ARCH "x86") elseif(${CMAKE_SYSTEM_PROCESSOR} STREQUAL "aarch64") @@ -289,6 +290,21 @@ set( mac-x86_64/crypto/test/trampoline-x86_64.S ) +set( + CRYPTO_win_aarch64_SOURCES + + win-aarch64/crypto/chacha/chacha-armv8.S + win-aarch64/crypto/fipsmodule/aesv8-armx64.S + win-aarch64/crypto/fipsmodule/armv8-mont.S + win-aarch64/crypto/fipsmodule/ghash-neon-armv8.S + win-aarch64/crypto/fipsmodule/ghashv8-armx64.S + win-aarch64/crypto/fipsmodule/sha1-armv8.S + win-aarch64/crypto/fipsmodule/sha256-armv8.S + win-aarch64/crypto/fipsmodule/sha512-armv8.S + win-aarch64/crypto/fipsmodule/vpaes-armv8.S + win-aarch64/crypto/test/trampoline-armv8.S +) + set( CRYPTO_win_x86_SOURCES @@ -331,9 +347,9 @@ set( win-x86_64/crypto/test/trampoline-x86_64.asm ) -if(APPLE AND ${ARCH} STREQUAL "aarch64") +if(APPLE AND ARCH STREQUAL "aarch64") set(CRYPTO_ARCH_SOURCES ${CRYPTO_ios_aarch64_SOURCES}) -elseif(APPLE AND ${ARCH} STREQUAL "arm") +elseif(APPLE AND ARCH STREQUAL "arm") set(CRYPTO_ARCH_SOURCES ${CRYPTO_ios_arm_SOURCES}) elseif(APPLE) set(CRYPTO_ARCH_SOURCES ${CRYPTO_mac_${ARCH}_SOURCES}) @@ -360,6 +376,7 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_object.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_octet.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_print.c" + "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_strex.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_strnid.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_time.c" "${BORINGSSL_SOURCE_DIR}/crypto/asn1/a_type.c" @@ -389,6 +406,7 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/bio/printf.c" "${BORINGSSL_SOURCE_DIR}/crypto/bio/socket.c" "${BORINGSSL_SOURCE_DIR}/crypto/bio/socket_helper.c" + "${BORINGSSL_SOURCE_DIR}/crypto/blake2/blake2.c" "${BORINGSSL_SOURCE_DIR}/crypto/bn_extra/bn_asn1.c" "${BORINGSSL_SOURCE_DIR}/crypto/bn_extra/convert.c" "${BORINGSSL_SOURCE_DIR}/crypto/buf/buf.c" @@ -413,6 +431,7 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/conf/conf.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-aarch64-fuchsia.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-aarch64-linux.c" + "${BORINGSSL_SOURCE_DIR}/crypto/cpu-aarch64-win.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-arm-linux.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-arm.c" "${BORINGSSL_SOURCE_DIR}/crypto/cpu-intel.c" @@ -452,7 +471,6 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/ex_data.c" "${BORINGSSL_SOURCE_DIR}/crypto/fipsmodule/bcm.c" "${BORINGSSL_SOURCE_DIR}/crypto/fipsmodule/fips_shared_support.c" - "${BORINGSSL_SOURCE_DIR}/crypto/fipsmodule/is_fips.c" "${BORINGSSL_SOURCE_DIR}/crypto/hkdf/hkdf.c" "${BORINGSSL_SOURCE_DIR}/crypto/hpke/hpke.c" "${BORINGSSL_SOURCE_DIR}/crypto/hrss/hrss.c" @@ -499,13 +517,13 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/trust_token/voprf.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_digest.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_sign.c" - "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_strex.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/a_verify.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/algorithm.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/asn1_gen.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/by_dir.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/by_file.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/i2d_pr.c" + "${BORINGSSL_SOURCE_DIR}/crypto/x509/name_print.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/rsa_pss.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/t_crl.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/t_req.c" @@ -519,7 +537,6 @@ add_library( "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_ext.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_lu.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_obj.c" - "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_r2x.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_req.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_set.c" "${BORINGSSL_SOURCE_DIR}/crypto/x509/x509_trs.c" @@ -589,6 +606,8 @@ add_library( "${BORINGSSL_SOURCE_DIR}/ssl/d1_srtp.cc" "${BORINGSSL_SOURCE_DIR}/ssl/dtls_method.cc" "${BORINGSSL_SOURCE_DIR}/ssl/dtls_record.cc" + "${BORINGSSL_SOURCE_DIR}/ssl/encrypted_client_hello.cc" + "${BORINGSSL_SOURCE_DIR}/ssl/extensions.cc" "${BORINGSSL_SOURCE_DIR}/ssl/handoff.cc" "${BORINGSSL_SOURCE_DIR}/ssl/handshake.cc" "${BORINGSSL_SOURCE_DIR}/ssl/handshake_client.cc" @@ -611,7 +630,6 @@ add_library( "${BORINGSSL_SOURCE_DIR}/ssl/ssl_versions.cc" "${BORINGSSL_SOURCE_DIR}/ssl/ssl_x509.cc" "${BORINGSSL_SOURCE_DIR}/ssl/t1_enc.cc" - "${BORINGSSL_SOURCE_DIR}/ssl/t1_lib.cc" "${BORINGSSL_SOURCE_DIR}/ssl/tls13_both.cc" "${BORINGSSL_SOURCE_DIR}/ssl/tls13_client.cc" "${BORINGSSL_SOURCE_DIR}/ssl/tls13_enc.cc" @@ -633,6 +651,7 @@ add_executable( "${BORINGSSL_SOURCE_DIR}/tool/digest.cc" "${BORINGSSL_SOURCE_DIR}/tool/fd.cc" "${BORINGSSL_SOURCE_DIR}/tool/file.cc" + "${BORINGSSL_SOURCE_DIR}/tool/generate_ech.cc" "${BORINGSSL_SOURCE_DIR}/tool/generate_ed25519.cc" "${BORINGSSL_SOURCE_DIR}/tool/genrsa.cc" "${BORINGSSL_SOURCE_DIR}/tool/pkcs12.cc" From 0d1fcdf9fcf1e6717c671f78c529fc331048a86b Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Mon, 11 Oct 2021 15:08:40 +0300 Subject: [PATCH 427/950] add submodule update --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index a6a2e2ab3e4..4c787e9d70c 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit a6a2e2ab3e44d97ce98e51c558e989f211de7eb3 +Subproject commit 4c787e9d70c370d51baea714e7b73910be2a4c28 From f4269ce41750648e3be629b08ce73e38afa273ae Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 8 Sep 2021 20:10:49 +0800 Subject: [PATCH 428/950] Allow optimize_arithmetic_operations_in_aggregate_functions when alias is used. --- .../ArithmeticOperationsInAgrFuncOptimize.cpp | 12 +++++++----- ...etic_operations_in_aggr_func_with_alias.reference | 10 ++++++++++ ...arithmetic_operations_in_aggr_func_with_alias.sql | 4 ++++ .../0_stateless/01470_columns_transformers.reference | 4 ++-- 4 files changed, 23 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.reference create mode 100644 tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql diff --git a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp index bdd9ce32707..70a58971d3f 100644 --- a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp +++ b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.cpp @@ -107,10 +107,7 @@ ASTPtr tryExchangeFunctions(const ASTFunction & func) || !supported.find(lower_name)->second.count(child_func->name)) return {}; - /// Cannot rewrite function with alias cause alias could become undefined - if (!func.tryGetAlias().empty() || !child_func->tryGetAlias().empty()) - return {}; - + auto original_alias = func.tryGetAlias(); const auto & child_func_args = child_func->arguments->children; const auto * first_literal = child_func_args[0]->as(); const auto * second_literal = child_func_args[1]->as(); @@ -132,7 +129,12 @@ ASTPtr tryExchangeFunctions(const ASTFunction & func) optimized_ast = exchangeExtractSecondArgument(new_name, *child_func); } - return optimized_ast; + if (optimized_ast) + { + optimized_ast->setAlias(original_alias); + return optimized_ast; + } + return {}; } } diff --git a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.reference b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.reference new file mode 100644 index 00000000000..9e0d871041b --- /dev/null +++ b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.reference @@ -0,0 +1,10 @@ +SELECT min(n AS a) + (1 AS b) AS c +FROM +( + SELECT number AS n + FROM numbers(10) + WHERE (1 > 0) AND (n > 0) +) +WHERE (a > 0) AND (b > 0) +HAVING c > 0 +2 diff --git a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql new file mode 100644 index 00000000000..73b87817bb3 --- /dev/null +++ b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func_with_alias.sql @@ -0,0 +1,4 @@ +set optimize_arithmetic_operations_in_aggregate_functions = 1; + +explain syntax select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0; +select min((n as a) + (1 as b)) c from (select number n from numbers(10)) where a > 0 and b > 0 having c > 0; diff --git a/tests/queries/0_stateless/01470_columns_transformers.reference b/tests/queries/0_stateless/01470_columns_transformers.reference index ae0adb3ba60..8fa86582018 100644 --- a/tests/queries/0_stateless/01470_columns_transformers.reference +++ b/tests/queries/0_stateless/01470_columns_transformers.reference @@ -54,8 +54,8 @@ SELECT sum(k) FROM columns_transformers SELECT - avg(i + 1 AS i), - avg(j + 2 AS j), + avg(i) + 1, + avg(j) + 2, avg(k) FROM columns_transformers SELECT From 635783fb663267180f25e92637b7915399b884b6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 24 Sep 2021 00:23:17 +0800 Subject: [PATCH 429/950] Only do TreeOptimizer for initial queries --- src/Interpreters/TreeOptimizer.cpp | 4 ---- src/Interpreters/TreeRewriter.cpp | 8 +++++++- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 3236418fe6f..8fb72f74c65 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -710,9 +709,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, if (settings.optimize_arithmetic_operations_in_aggregate_functions) optimizeAggregationFunctions(query); - /// Push the predicate expression down to the subqueries. - result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); - /// GROUP BY injective function elimination. optimizeGroupBy(select_query, result.source_columns_set, context); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8f923d82b27..9bcddb6b982 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -1036,7 +1037,12 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); - TreeOptimizer::apply(query, result, tables_with_columns, getContext()); + /// Push the predicate expression down to subqueries. The optimization should be applied to both initial and secondary queries. + result.rewrite_subqueries = PredicateExpressionsOptimizer(getContext(), tables_with_columns, settings).optimize(*select_query); + + /// Only apply AST optimization for initial queries. + if (getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + TreeOptimizer::apply(query, result, tables_with_columns, getContext()); /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); From 3ae960e04b7675b54d5e386573c387c72ad1e5cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Oct 2021 16:40:12 +0300 Subject: [PATCH 430/950] Review fixes --- docker/test/stress/stress | 2 +- tests/clickhouse-test | 6 +++--- ...4_cancel_http_readonly_queries_on_client_close.sh | 5 +++-- .../0_stateless/01085_max_distributed_connections.sh | 5 +++-- .../01085_max_distributed_connections_http.sh | 5 +++-- .../0_stateless/01600_quota_by_forwarded_ip.sh | 12 +++++++++--- .../0_stateless/01602_max_distributed_connections.sh | 8 ++++++-- .../queries/0_stateless/01675_data_type_coroutine.sh | 6 ++++-- .../0_stateless/01681_hyperscan_debug_assertion.sh | 5 +++-- .../01834_alias_columns_laziness_filimonov.sh | 5 +++-- tests/queries/0_stateless/02044_url_glob_parallel.sh | 6 ++++-- 11 files changed, 42 insertions(+), 23 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 1559b084565..5e98c67d8e1 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -117,7 +117,7 @@ def prepare_for_hung_check(drop_databases): # Wait for last queries to finish if any, not longer than 300 seconds call("""clickhouse client -q "select sleepEachRow(( select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300 - ) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT, timeout=30) + ) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT, timeout=330) # Even if all clickhouse-test processes are finished, there are probably some sh scripts, # which still run some new queries. Let's ignore them. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 061333297e2..62860a36fc7 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -879,7 +879,7 @@ def run_tests_array(all_tests_with_params): while True: if is_concurrent: - case = queue.get(timeout=args.timeout) + case = queue.get(timeout=args.timeout * 1.1) if not case: break else: @@ -1080,10 +1080,10 @@ def do_run_tests(jobs, test_suite: TestSuite, parallel): pool.map_async(run_tests_array, parallel_tests_array) for suit in test_suite.parallel_tests: - queue.put(suit, timeout=args.timeout) + queue.put(suit, timeout=args.timeout * 1.1) for _ in range(jobs): - queue.put(None, timeout=args.timeout) + queue.put(None, timeout=args.timeout * 1.1) queue.close() diff --git a/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh b/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh index 340df58e473..74b4c4052f8 100755 --- a/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh +++ b/tests/queries/0_stateless/00834_cancel_http_readonly_queries_on_client_close.sh @@ -7,9 +7,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CURL} --max-time 1 -sS "${CLICKHOUSE_URL}&query_id=cancel_http_readonly_queries_on_client_close&cancel_http_readonly_queries_on_client_close=1&query=SELECT+count()+FROM+system.numbers" 2>&1 | grep -cF 'curl: (28)' -while true -do +i=0 retries=300 +while [[ $i -lt $retries ]]; do ${CLICKHOUSE_CURL} -sS --data "SELECT count() FROM system.processes WHERE query_id = 'cancel_http_readonly_queries_on_client_close'" "${CLICKHOUSE_URL}" | grep '0' && break + ((++i)) sleep 0.2 done diff --git a/tests/queries/0_stateless/01085_max_distributed_connections.sh b/tests/queries/0_stateless/01085_max_distributed_connections.sh index 4ffcd980956..34862289d1e 100755 --- a/tests/queries/0_stateless/01085_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01085_max_distributed_connections.sh @@ -5,10 +5,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +i=0 retries=300 # Sometimes five seconds are not enough due to system overload. # But if it can run in less than five seconds at least sometimes - it is enough for the test. -while true -do +while [[ $i -lt $retries ]]; do opts=( --max_distributed_connections 20 --max_threads 1 @@ -19,4 +19,5 @@ do # "$@" left to pass manual options (like --experimental_use_processors 0) during manual testing timeout 10s ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" && break + ((++i)) done diff --git a/tests/queries/0_stateless/01085_max_distributed_connections_http.sh b/tests/queries/0_stateless/01085_max_distributed_connections_http.sh index 3edf70f31b8..0e40918257d 100755 --- a/tests/queries/0_stateless/01085_max_distributed_connections_http.sh +++ b/tests/queries/0_stateless/01085_max_distributed_connections_http.sh @@ -8,9 +8,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Sometimes 1.8 seconds are not enough due to system overload. # But if it can run in less than five seconds at least sometimes - it is enough for the test. -while true -do +i=0 retries=100 +while [[ $i -lt $retries ]]; do query="SELECT sleepEachRow(1) FROM remote('127.{2,3}', system.one) FORMAT Null" # 1.8 less then 2 seconds, but long enough to cover possible load peaks timeout 1.8s ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&max_distributed_connections=2&max_threads=1" -d "$query" && break + ((++i)) done diff --git a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh index 97e4da5f9e3..1d768c8b027 100755 --- a/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh +++ b/tests/queries/0_stateless/01600_quota_by_forwarded_ip.sh @@ -21,7 +21,9 @@ CREATE QUOTA quota_by_forwarded_ip_${CLICKHOUSE_DATABASE} KEYED BY forwarded_ip_ echo '--- Test with quota by immediate IP ---' -while true; do +i=0 retries=300 +while [[ $i -lt $retries ]]; do + ((++i)) ${CLICKHOUSE_CURL} --fail -sS "${CLICKHOUSE_URL}&user=quoted_by_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" 2>/dev/null || break done | uniq @@ -33,14 +35,18 @@ ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quo echo '--- Test with quota by forwarded IP ---' -while true; do +i=0 retries=300 +while [[ $i -lt $retries ]]; do + ((++i)) ${CLICKHOUSE_CURL} --fail -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" 2>/dev/null || break done | uniq ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oF 'exceeded' +i=0 retries=300 # X-Forwarded-For is respected for quota by forwarded IP address -while true; do +while [[ $i -lt $retries ]]; do + ((++i)) ${CLICKHOUSE_CURL} -H 'X-Forwarded-For: 1.2.3.4' -sS "${CLICKHOUSE_URL}&user=quoted_by_forwarded_ip_${CLICKHOUSE_DATABASE}" -d "SELECT count() FROM numbers(10)" | grep -oP '^10$' || break done | uniq diff --git a/tests/queries/0_stateless/01602_max_distributed_connections.sh b/tests/queries/0_stateless/01602_max_distributed_connections.sh index 51ff803ad5e..ed835a8768f 100755 --- a/tests/queries/0_stateless/01602_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01602_max_distributed_connections.sh @@ -13,14 +13,18 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # If concurrency is 10 (good), the query may take less than 10 second with non-zero probability # and the following loops will finish with probability 1 assuming independent random variables. -while true; do +i=0 retries=30 +while [[ $i -lt $retries ]]; do timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && break + ((++i)) done -while true; do +i=0 retries=30 +while [[ $i -lt $retries ]]; do timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && break + ((++i)) done # If max_distributed_connections is low and async_socket_for_remote is disabled, diff --git a/tests/queries/0_stateless/01675_data_type_coroutine.sh b/tests/queries/0_stateless/01675_data_type_coroutine.sh index 781e43e4134..8e80d722a4c 100755 --- a/tests/queries/0_stateless/01675_data_type_coroutine.sh +++ b/tests/queries/0_stateless/01675_data_type_coroutine.sh @@ -4,12 +4,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh + +counter=0 retries=60 I=0 -while true -do +while [[ $counter -lt $retries ]]; do I=$((I + 1)) TYPE=$(perl -e "print 'Array(' x $I; print 'UInt8'; print ')' x $I") ${CLICKHOUSE_CLIENT} --max_parser_depth 1000000 --query "SELECT * FROM remote('127.0.0.{1,2}', generateRandom('x $TYPE', 1, 1, 1)) LIMIT 1 FORMAT Null" 2>&1 | grep -q -F 'Maximum parse depth' && break; + ((++counter)) done #echo "I = ${I}" diff --git a/tests/queries/0_stateless/01681_hyperscan_debug_assertion.sh b/tests/queries/0_stateless/01681_hyperscan_debug_assertion.sh index 2b4cd1a5f01..62469da0b3e 100755 --- a/tests/queries/0_stateless/01681_hyperscan_debug_assertion.sh +++ b/tests/queries/0_stateless/01681_hyperscan_debug_assertion.sh @@ -13,13 +13,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) M=1000000 -while true -do +i=0 retries=300 +while [[ $i -lt $retries ]]; do $CLICKHOUSE_CLIENT --allow_hyperscan 1 --max_memory_usage $M --format Null --query " SELECT [1, 2, 3, 11] = arraySort(multiMatchAllIndices('фабрикант', ['', 'рикан', 'а', 'f[a${RANDOM}e]b[ei]rl', 'ф[иа${RANDOM}эе]б[еэи][рпл]', 'афиукд', 'a[f${RANDOM}t],th', '^ф[аие${RANDOM}э]?б?[еэи]?$', 'бе${RANDOM}рлик', 'fa${RANDOM}b', 'фа[беьв]+е?[рл${RANDOM}ко]'])) " 2>&1 | grep -q 'Memory limit' || break; M=$((M + 100000)) + ((++i)) done echo 'Ok' diff --git a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh index 793f477b3cb..1d70ba1df7c 100755 --- a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh +++ b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.sh @@ -16,9 +16,10 @@ insert into aliases_lazyness(x) select * from numbers(40); # The exact time is not guaranteed, so we check in a loop that at least once # the query will process in less than one second, that proves that the behaviour is not like it was long time ago. -while true -do +i=0 retries=300 +while [[ $i -lt $retries ]]; do timeout 1 ${CLICKHOUSE_CLIENT} --query "SELECT x, y FROM aliases_lazyness WHERE x = 1 FORMAT Null" && break + ((++i)) done ${CLICKHOUSE_CLIENT} --multiquery --query " diff --git a/tests/queries/0_stateless/02044_url_glob_parallel.sh b/tests/queries/0_stateless/02044_url_glob_parallel.sh index 6491a661201..c9c779a9ddb 100755 --- a/tests/queries/0_stateless/02044_url_glob_parallel.sh +++ b/tests/queries/0_stateless/02044_url_glob_parallel.sh @@ -5,9 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh + +i=0 retries=60 # Sometimes five seconds are not enough due to system overload. # But if it can run in less than five seconds at least sometimes - it is enough for the test. -while true -do +while [[ $i -lt $retries ]]; do timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --query "SELECT * FROM url('http://127.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null && break + ((++i)) done From ab4b2295749a900c803dd8620d19f8d46a7d023d Mon Sep 17 00:00:00 2001 From: Haavard Kvaalen Date: Mon, 11 Oct 2021 16:20:51 +0200 Subject: [PATCH 431/950] Make sure we update position on commit Make sure we update GTID set on QueryEvents with "COMMIT" or "XA COMMIT". Without this we could have to redo the last transaction if e.g. ClickHouse was restarted. Note that this did not affect normal transactions on InnoDB, since they are terminated with a XID_EVENT. --- src/Core/MySQL/MySQLReplication.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index b5468d15edc..b5adab67e3a 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -815,6 +815,7 @@ namespace MySQLReplication { event = std::make_shared(std::move(event_header)); event->parseEvent(event_payload); + position.update(event); auto query = std::static_pointer_cast(event); switch (query->typ) @@ -826,7 +827,7 @@ namespace MySQLReplication break; } default: - position.update(event); + break; } break; } From 362bcb2f6662c7c05731efed229d0a9273bc307f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 30 Aug 2021 14:04:59 +0300 Subject: [PATCH 432/950] Introduce ProfileEvents packet --- programs/client/Client.cpp | 4 + src/Client/ClientBase.cpp | 9 ++ src/Client/ClientBase.h | 1 + src/Client/Connection.cpp | 21 ++++ src/Client/Connection.h | 3 + src/Client/HedgedConnections.cpp | 2 + src/Client/LocalConnection.cpp | 2 + src/Client/MultiplexedConnections.cpp | 2 + src/Client/Suggest.cpp | 3 + src/Common/CurrentMetrics.h | 6 ++ src/Common/MemoryTracker.h | 5 + src/Core/Protocol.h | 3 +- src/DataStreams/ConnectionCollector.cpp | 2 + src/DataStreams/RemoteQueryExecutor.cpp | 4 + src/Server/TCPHandler.cpp | 137 ++++++++++++++++++++++++ src/Server/TCPHandler.h | 4 + 16 files changed, 207 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 04193036872..da910430985 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -15,6 +15,7 @@ #include #include #include "Client.h" +#include "Core/Protocol.h" #include #include @@ -377,6 +378,9 @@ std::vector Client::loadWarningMessages() case Protocol::Server::EndOfStream: return messages; + case Protocol::Server::ProfileEvents: + continue; + default: throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 56d9993d14b..ee5f3580050 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Core/Protocol.h" #if !defined(ARCADIA_BUILD) # include @@ -611,6 +612,10 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled) onEndOfStream(); return false; + case Protocol::Server::ProfileEvents: + onProfileEvents(); + return true; + default: throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}", packet.type, connection->getDescription()); @@ -651,6 +656,10 @@ void ClientBase::onEndOfStream() } +void ClientBase::onProfileEvents() +{} + + /// Flush all buffers. void ClientBase::resetOutput() { diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index b122803e1db..0fa205a4d6e 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -114,6 +114,7 @@ private: void onReceiveExceptionFromServer(std::unique_ptr && e); void onProfileInfo(const BlockStreamProfileInfo & profile_info); void onEndOfStream(); + void onProfileEvents(); void sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query); void sendDataFrom(ReadBuffer & buf, Block & sample, diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2aa157bb318..1aabe449ed5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -870,6 +871,10 @@ Packet Connection::receivePacket() case Protocol::Server::ReadTaskRequest: return res; + case Protocol::Server::ProfileEvents: + res.block = receiveProfileEvents(); + return res; + default: /// In unknown state, disconnect - to not leave unsynchronised connection. disconnect(); @@ -923,6 +928,13 @@ Block Connection::receiveDataImpl(NativeReader & reader) } +Block Connection::receiveProfileEvents() +{ + initBlockProfileEventsInput(); + return receiveDataImpl(*block_profile_events_in); +} + + void Connection::initInputBuffers() { @@ -956,6 +968,15 @@ void Connection::initBlockLogsInput() } +void Connection::initBlockProfileEventsInput() +{ + if (!block_profile_events_in) + { + block_profile_events_in = std::make_unique(*in, server_revision); + } +} + + void Connection::setDescription() { auto resolved_address = getResolvedAddress(); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index a5130d876ea..b6054941aeb 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -206,6 +206,7 @@ private: std::shared_ptr maybe_compressed_in; std::unique_ptr block_in; std::unique_ptr block_logs_in; + std::unique_ptr block_profile_events_in; /// Where to write data for INSERT. std::shared_ptr maybe_compressed_out; @@ -249,6 +250,7 @@ private: Block receiveData(); Block receiveLogData(); Block receiveDataImpl(NativeReader & reader); + Block receiveProfileEvents(); std::vector receiveMultistringMessage(UInt64 msg_type) const; std::unique_ptr receiveException() const; @@ -258,6 +260,7 @@ private: void initInputBuffers(); void initBlockInput(); void initBlockLogsInput(); + void initBlockProfileEventsInput(); [[noreturn]] void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const; }; diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index b833241b2bc..1ca890f40f9 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -1,3 +1,4 @@ +#include "Core/Protocol.h" #if defined(OS_LINUX) #include @@ -412,6 +413,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: + case Protocol::Server::ProfileEvents: replica_with_last_received_packet = replica_location; break; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 29bc0c84437..efd302622dd 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -5,6 +5,7 @@ #include #include #include +#include "Core/Protocol.h" namespace DB @@ -328,6 +329,7 @@ Packet LocalConnection::receivePacket() case Protocol::Server::Extremes: [[fallthrough]]; case Protocol::Server::Log: [[fallthrough]]; case Protocol::Server::Data: + case Protocol::Server::ProfileEvents: { if (state->block && state->block.value()) { diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index a4e1eb09253..a27f7709555 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -2,6 +2,7 @@ #include #include #include +#include "Core/Protocol.h" namespace DB @@ -320,6 +321,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: + case Protocol::Server::ProfileEvents: break; case Protocol::Server::EndOfStream: diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index f500332b616..38aeae76a38 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "Core/Protocol.h" #include #include #include @@ -162,6 +163,8 @@ void Suggest::fetch(IServerConnection & connection, const ConnectionTimeouts & t continue; case Protocol::Server::Log: continue; + case Protocol::Server::ProfileEvents: + continue; case Protocol::Server::Exception: packet.exception->rethrow(); diff --git a/src/Common/CurrentMetrics.h b/src/Common/CurrentMetrics.h index f6f4785a95a..21c3f704872 100644 --- a/src/Common/CurrentMetrics.h +++ b/src/Common/CurrentMetrics.h @@ -41,6 +41,12 @@ namespace CurrentMetrics values[metric].store(value, std::memory_order_relaxed); } + /// Get value of specified metric. + inline Value get(Metric metric) + { + return values[metric].load(std::memory_order_relaxed); + } + /// Add value for specified metric. You must subtract value later; or see class Increment below. inline void add(Metric metric, Value value = 1) { diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index b860c611be2..7da70db0876 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -143,6 +143,11 @@ public: metric.store(metric_, std::memory_order_relaxed); } + CurrentMetrics::Metric getMetric() + { + return metric.load(std::memory_order_relaxed); + } + void setDescription(const char * description) { description_ptr.store(description, std::memory_order_relaxed); diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 9ec792a6230..4958f343bbc 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -80,7 +80,8 @@ namespace Protocol ReadTaskRequest = 13, /// String (UUID) describes a request for which next task is needed /// This is such an inverted logic, where server sends requests /// And client returns back response - MAX = ReadTaskRequest, + ProfileEvents = 14, + MAX = ProfileEvents, }; /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 diff --git a/src/DataStreams/ConnectionCollector.cpp b/src/DataStreams/ConnectionCollector.cpp index 8e700c0ab7f..df206478e91 100644 --- a/src/DataStreams/ConnectionCollector.cpp +++ b/src/DataStreams/ConnectionCollector.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Core/Protocol.h" #include namespace CurrentMetrics @@ -81,6 +82,7 @@ void ConnectionCollector::drainConnections(IConnections & connections) noexcept { case Protocol::Server::EndOfStream: case Protocol::Server::Log: + case Protocol::Server::ProfileEvents: break; case Protocol::Server::Exception: diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 3c78fddfd39..fc2db2f3f6f 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -4,6 +4,7 @@ #include #include +#include "Core/Protocol.h" #include #include #include @@ -390,6 +391,9 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) log_queue->pushBlock(std::move(packet.block)); break; + case Protocol::Server::ProfileEvents: + break; + default: got_unknown_packet_from_replica = true; throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from one of the following replicas: {}", diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f3247e7bc2b..fbb5c755142 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -812,6 +813,128 @@ void TCPHandler::sendExtremes(const Block & extremes) } +namespace +{ + using namespace ProfileEvents; + + enum ProfileEventTypes : int8_t + { + INCREMENT = 1, + GAUGE = 2, + }; + + constexpr size_t NAME_COLUMN_INDEX = 4; + constexpr size_t VALUE_COLUMN_INDEX = 5; + + /* + * Add records about provided non-zero ProfileEvents::Counters. + */ + void dumpProfileEvents( + ProfileEvents::Counters const & snapshot, + MutableColumns & columns, + String const & host_name, + time_t current_time, + UInt64 thread_id) + { + size_t rows = 0; + auto & name_column = columns[NAME_COLUMN_INDEX]; + auto & value_column = columns[VALUE_COLUMN_INDEX]; + for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) + { + UInt64 value = snapshot[event].load(std::memory_order_relaxed); + + if (value == 0) + continue; + + const char * desc = ProfileEvents::getName(event); + name_column->insertData(desc, strlen(desc)); + value_column->insert(value); + rows++; + } + + // Fill the rest of the columns with data + for (size_t row = 0; row < rows; ++row) + { + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(current_time)); + columns[i++]->insert(UInt64{thread_id}); + columns[i++]->insert(ProfileEventTypes::INCREMENT); + } + } + + void dumpMemoryTracker( + MemoryTracker * memoryTracker, + MutableColumns & columns, + String const & host_name, + time_t current_time, + UInt64 thread_id) + { + auto metric = memoryTracker->getMetric(); + if (metric == CurrentMetrics::end()) + return; + + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(current_time)); + columns[i++]->insert(UInt64{thread_id}); + columns[i++]->insert(ProfileEventTypes::GAUGE); + + auto const * metric_name = CurrentMetrics::getName(metric); + columns[i++]->insertData(metric_name, strlen(metric_name)); + auto metric_value = CurrentMetrics::get(metric); + columns[i++]->insert(metric_value); + } +} + + +void TCPHandler::sendProfileEvents() +{ + auto thread_group = CurrentThread::getGroup(); + auto const counters_snapshot = CurrentThread::getProfileEvents().getPartiallyAtomicSnapshot(); + auto current_time = time(nullptr); + auto * memory_tracker = CurrentThread::getMemoryTracker(); + + auto const thread_id = CurrentThread::get().thread_id; + + auto profile_event_type = std::make_shared( + DataTypeEnum8::Values + { + { "increment", static_cast(INCREMENT)}, + { "gauge", static_cast(GAUGE)}, + }); + + NamesAndTypesList column_names_and_types = { + { "host_name", std::make_shared() }, + { "current_time", std::make_shared() }, + { "thread_id", std::make_shared() }, + { "type", profile_event_type }, + { "name", std::make_shared() }, + { "value", std::make_shared() }, + }; + + ColumnsWithTypeAndName temp_columns; + for (auto const & name_and_type : column_names_and_types) + temp_columns.emplace_back(name_and_type.type, name_and_type.name); + + Block block(std::move(temp_columns)); + + MutableColumns columns = block.mutateColumns(); + dumpProfileEvents(counters_snapshot, columns, server_display_name, current_time, thread_id); + dumpMemoryTracker(memory_tracker, columns, server_display_name, current_time, thread_id); + + block.setColumns(std::move(columns)); + + initProfileEventsBlockOutput(block); + + writeVarUInt(Protocol::Server::ProfileEvents, *out); + writeStringBinary("", *out); + + state.logs_block_out->write(block); + out->next(); +} + + bool TCPHandler::receiveProxyHeader() { if (in->eof()) @@ -1453,6 +1576,20 @@ void TCPHandler::initLogsBlockOutput(const Block & block) } +void TCPHandler::initProfileEventsBlockOutput(const Block & block) +{ + if (!state.profile_events_block_out) + { + const Settings & query_settings = query_context->getSettingsRef(); + state.profile_events_block_out = std::make_unique( + *out, + client_tcp_protocol_version, + block.cloneEmpty(), + !query_settings.low_cardinality_allow_in_native_format); + } +} + + bool TCPHandler::isQueryCancelled() { if (state.is_cancelled || state.sent_all_data) diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index d001b12ee66..9ff061e096b 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -48,6 +48,8 @@ struct QueryState InternalTextLogsQueuePtr logs_queue; std::unique_ptr logs_block_out; + std::unique_ptr profile_events_block_out; + /// From where to read data for INSERT. std::shared_ptr maybe_compressed_in; std::unique_ptr block_in; @@ -228,11 +230,13 @@ private: void sendProfileInfo(const BlockStreamProfileInfo & info); void sendTotals(const Block & totals); void sendExtremes(const Block & extremes); + void sendProfileEvents(); /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); void initBlockOutput(const Block & block); void initLogsBlockOutput(const Block & block); + void initProfileEventsBlockOutput(const Block & block); bool isQueryCancelled(); From e9b1e0546179c38fe748b5df947e96bcd95771b4 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 30 Aug 2021 18:35:25 +0300 Subject: [PATCH 433/950] Send profile events from all threads of current group --- src/Common/ThreadStatus.h | 4 +++- src/Interpreters/ThreadStatusExt.cpp | 1 + src/Server/TCPHandler.cpp | 23 +++++++++++++---------- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 9e8d8f637b8..dbd0b4e5664 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -15,6 +15,7 @@ #include #include #include +#include namespace Poco @@ -41,7 +42,7 @@ struct ViewRuntimeData; class QueryViewsLog; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; - +using ThreadStatusPtr = ThreadStatus *; /** Thread group is a collection of threads dedicated to single task * (query or other process like background merge). @@ -66,6 +67,7 @@ public: std::function fatal_error_callback; std::vector thread_ids; + std::unordered_set threads; /// The first thread created this thread group UInt64 master_thread_id = 0; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 465b8e31b08..81a745ef430 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -123,6 +123,7 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) /// NOTE: thread may be attached multiple times if it is reused from a thread pool. thread_group->thread_ids.emplace_back(thread_id); + thread_group->threads.insert(this); logs_queue_ptr = thread_group->logs_queue_ptr; fatal_error_callback = thread_group->fatal_error_callback; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index fbb5c755142..4c6d01c564c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -666,6 +666,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() /// Some time passed and there is a progress. after_send_progress.restart(); sendProgress(); + sendProfileEvents(); } sendLogs(); @@ -691,6 +692,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() sendProfileInfo(executor.getProfileInfo()); sendProgress(); sendLogs(); + sendProfileEvents(); } if (state.is_connection_closed) @@ -867,12 +869,12 @@ namespace MemoryTracker * memoryTracker, MutableColumns & columns, String const & host_name, - time_t current_time, UInt64 thread_id) { auto metric = memoryTracker->getMetric(); if (metric == CurrentMetrics::end()) return; + time_t current_time = time(nullptr); size_t i = 0; columns[i++]->insertData(host_name.data(), host_name.size()); @@ -890,13 +892,6 @@ namespace void TCPHandler::sendProfileEvents() { - auto thread_group = CurrentThread::getGroup(); - auto const counters_snapshot = CurrentThread::getProfileEvents().getPartiallyAtomicSnapshot(); - auto current_time = time(nullptr); - auto * memory_tracker = CurrentThread::getMemoryTracker(); - - auto const thread_id = CurrentThread::get().thread_id; - auto profile_event_type = std::make_shared( DataTypeEnum8::Values { @@ -920,9 +915,17 @@ void TCPHandler::sendProfileEvents() Block block(std::move(temp_columns)); MutableColumns columns = block.mutateColumns(); - dumpProfileEvents(counters_snapshot, columns, server_display_name, current_time, thread_id); - dumpMemoryTracker(memory_tracker, columns, server_display_name, current_time, thread_id); + auto thread_group = CurrentThread::getGroup(); + for (auto * thread : thread_group->threads) + { + auto const counters_snapshot = thread->performance_counters.getPartiallyAtomicSnapshot(); + auto current_time = time(nullptr); + auto * memory_tracker = &thread->memory_tracker; + auto const thread_id = CurrentThread::get().thread_id; + dumpProfileEvents(counters_snapshot, columns, server_display_name, current_time, thread_id); + dumpMemoryTracker(memory_tracker, columns, server_display_name, thread_id); + } block.setColumns(std::move(columns)); initProfileEventsBlockOutput(block); From 4c5a77457076dff8041c3baaeb89feef6c853d6f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 30 Aug 2021 18:48:22 +0300 Subject: [PATCH 434/950] Add comment --- src/Core/Protocol.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 4958f343bbc..b2957e4ae30 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -80,7 +80,7 @@ namespace Protocol ReadTaskRequest = 13, /// String (UUID) describes a request for which next task is needed /// This is such an inverted logic, where server sends requests /// And client returns back response - ProfileEvents = 14, + ProfileEvents = 14, /// Packet with profile events from server. MAX = ProfileEvents, }; From 803b8623c1b03dcb5bd4591d67dea421cd121b2e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 31 Aug 2021 16:50:56 +0300 Subject: [PATCH 435/950] Fix TCPHandler::sendProfileEvents --- src/Interpreters/ThreadStatusExt.cpp | 5 +++++ src/Server/TCPHandler.cpp | 13 +++++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 81a745ef430..7b7bfec006c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -398,6 +399,10 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) finalizePerformanceCounters(); /// Detach from thread group + { + std::lock_guard guard(thread_group->mutex); + thread_group->threads.erase(this); + } performance_counters.setParent(&ProfileEvents::global_counters); memory_tracker.reset(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 4c6d01c564c..9c1b107c513 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,4 +1,8 @@ +#include #include +#include +#include +#include #include #include #include @@ -916,7 +920,12 @@ void TCPHandler::sendProfileEvents() MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); - for (auto * thread : thread_group->threads) + std::vector threads; + { + std::lock_guard guard(thread_group->mutex); + std::copy(thread_group->threads.begin(), thread_group->threads.end(), std::back_inserter(threads)); + } + for (auto * thread : threads) { auto const counters_snapshot = thread->performance_counters.getPartiallyAtomicSnapshot(); auto current_time = time(nullptr); @@ -933,7 +942,7 @@ void TCPHandler::sendProfileEvents() writeVarUInt(Protocol::Server::ProfileEvents, *out); writeStringBinary("", *out); - state.logs_block_out->write(block); + state.profile_events_block_out->write(block); out->next(); } From 74cdaba7fa90bf0e87c24c7c6a6c384249f33d5c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 1 Sep 2021 17:47:12 +0300 Subject: [PATCH 436/950] WIP on profile events forwarding --- src/DataStreams/RemoteQueryExecutor.cpp | 1 + src/Server/GRPCServer.cpp | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index fc2db2f3f6f..dc97e577513 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -392,6 +392,7 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) break; case Protocol::Server::ProfileEvents: + /// Pass profile events from remote server to client break; default: diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index cc3c7085dfd..fc712916372 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -596,6 +596,7 @@ namespace void addExtremesToResult(const Block & extremes); void addProfileInfoToResult(const BlockStreamProfileInfo & info); void addLogsToResult(); + void addProfileEventsToResult(); void sendResult(); void throwIfFailedToSendResult(); void sendException(const Exception & exception); @@ -1123,6 +1124,7 @@ namespace if (after_send_progress.elapsedMicroseconds() >= interactive_delay) { addProgressToResult(); + addProfileEventsToResult(); after_send_progress.restart(); } @@ -1174,6 +1176,7 @@ namespace finalize = true; io.onFinish(); addProgressToResult(); + addProfileEventsToResult(); query_scope->logPeakMemoryUsage(); addLogsToResult(); sendResult(); @@ -1437,6 +1440,11 @@ namespace } } + void Call::addProfileEventsToResult() + { + + } + void Call::sendResult() { /// gRPC doesn't allow to write anything to a finished responder. From 356723427df530d834dd9f0bae1977b95f1ccc84 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 2 Sep 2021 17:27:19 +0300 Subject: [PATCH 437/950] WIP on ProfileEvents forwarding --- src/Client/Connection.cpp | 2 ++ src/Common/CurrentThread.cpp | 18 +++++++++++ src/Common/CurrentThread.h | 3 ++ src/Common/ThreadStatus.cpp | 12 ++++++++ src/Common/ThreadStatus.h | 15 +++++++++ src/Core/Protocol.h | 3 +- src/DataStreams/RemoteQueryExecutor.cpp | 9 +++++- src/Interpreters/ThreadStatusExt.cpp | 1 + src/Server/TCPHandler.cpp | 41 ++++++++++++++++++++----- src/Server/TCPHandler.h | 1 + 10 files changed, 95 insertions(+), 10 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 1aabe449ed5..c6badf96bf9 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -22,6 +22,7 @@ #include #include #include +#include "Core/Block.h" #include #include #include @@ -872,6 +873,7 @@ Packet Connection::receivePacket() return res; case Protocol::Server::ProfileEvents: + LOG_DEBUG(log_wrapper.get(), "Connection received ProfileEvents"); res.block = receiveProfileEvents(); return res; diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index c6b9e027c48..10d9f4d07df 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -91,6 +91,24 @@ std::shared_ptr CurrentThread::getInternalTextLogsQueue() return current_thread->getInternalTextLogsQueue(); } +void CurrentThread::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue) +{ + if (unlikely(!current_thread)) + return; + current_thread->attachInternalProfileEventsQueue(queue); +} + +InternalProfileEventsQueuePtr CurrentThread::getInternalProfileEventsQueue() +{ + if (unlikely(!current_thread)) + return nullptr; + + if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) + return nullptr; + + return current_thread->getInternalProfileEventsQueue(); +} + ThreadGroupStatusPtr CurrentThread::getGroup() { if (unlikely(!current_thread)) diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 96ea7f7e795..9dbe8d355d6 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -46,6 +46,9 @@ public: LogsLevel client_logs_level); static std::shared_ptr getInternalTextLogsQueue(); + static void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & queue); + static InternalProfileEventsQueuePtr getInternalProfileEventsQueue(); + static void setFatalErrorCallback(std::function callback); /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index b1d76c4660e..4c49e9b1d0d 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB @@ -197,6 +198,17 @@ void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & thread_group->client_logs_level = client_logs_level; } +void ThreadStatus::attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue) +{ + profile_queue_ptr = profile_queue; + + if (!thread_group) + return; + + std::lock_guard lock(thread_group->mutex); + thread_group->profile_queue_ptr = profile_queue; +} + void ThreadStatus::setFatalErrorCallback(std::function callback) { fatal_error_callback = std::move(callback); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index dbd0b4e5664..16a47a21184 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -42,6 +43,10 @@ struct ViewRuntimeData; class QueryViewsLog; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; + +using InternalProfileEventsQueue = ConcurrentBoundedQueue; +using InternalProfileEventsQueuePtr = std::shared_ptr; +using InternalProfileEventsQueueWeakPtr = std::weak_ptr; using ThreadStatusPtr = ThreadStatus *; /** Thread group is a collection of threads dedicated to single task @@ -64,6 +69,7 @@ public: ContextWeakPtr global_context; InternalTextLogsQueueWeakPtr logs_queue_ptr; + InternalProfileEventsQueueWeakPtr profile_queue_ptr; std::function fatal_error_callback; std::vector thread_ids; @@ -134,6 +140,8 @@ protected: /// A logs queue used by TCPHandler to pass logs to a client InternalTextLogsQueueWeakPtr logs_queue_ptr; + InternalProfileEventsQueueWeakPtr profile_queue_ptr; + bool performance_counters_finalized = false; UInt64 query_start_time_nanoseconds = 0; UInt64 query_start_time_microseconds = 0; @@ -208,6 +216,13 @@ public: void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue, LogsLevel client_logs_level); + InternalProfileEventsQueuePtr getInternalProfileEventsQueue() const + { + return thread_state == Died ? nullptr : profile_queue_ptr.lock(); + } + + void attachInternalProfileEventsQueue(const InternalProfileEventsQueuePtr & profile_queue); + /// Callback that is used to trigger sending fatal error messages to client. void setFatalErrorCallback(std::function callback); void onFatalError(); diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index b2957e4ae30..fb18e1135a5 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -104,7 +104,8 @@ namespace Protocol "Log", "TableColumns", "PartUUIDs", - "ReadTaskRequest" + "ReadTaskRequest", + "ProfileEvents", }; return packet <= MAX ? data[packet] diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index dc97e577513..51c5c2edc57 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -1,3 +1,5 @@ +#include + #include #include #include @@ -393,7 +395,12 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) case Protocol::Server::ProfileEvents: /// Pass profile events from remote server to client - break; + { + LOG_DEBUG(log, "RemoteQueryExecutor received ProfileEvents"); + auto profile_queue = CurrentThread::getInternalProfileEventsQueue(); + profile_queue->emplace(std::move(packet.block)); + break; + } default: got_unknown_packet_from_replica = true; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 7b7bfec006c..7ff74a0618c 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -129,6 +129,7 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) logs_queue_ptr = thread_group->logs_queue_ptr; fatal_error_callback = thread_group->fatal_error_callback; query_context = thread_group->query_context; + profile_queue_ptr = thread_group->profile_queue_ptr; if (global_context.expired()) global_context = thread_group->global_context; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9c1b107c513..f1014d611fd 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -243,6 +244,8 @@ void TCPHandler::runImpl() sendLogs(); }); } + state.profile_queue = std::make_shared(std::numeric_limits::max()); + CurrentThread::attachInternalProfileEventsQueue(state.profile_queue); query_context->setExternalTablesInitializer([this] (ContextPtr context) { @@ -670,10 +673,10 @@ void TCPHandler::processOrdinaryQueryWithProcessors() /// Some time passed and there is a progress. after_send_progress.restart(); sendProgress(); - sendProfileEvents(); } sendLogs(); + sendProfileEvents(); if (block) { @@ -696,7 +699,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() sendProfileInfo(executor.getProfileInfo()); sendProgress(); sendLogs(); - sendProfileEvents(); + // sendProfileEvents(); } if (state.is_connection_closed) @@ -935,15 +938,37 @@ void TCPHandler::sendProfileEvents() dumpProfileEvents(counters_snapshot, columns, server_display_name, current_time, thread_id); dumpMemoryTracker(memory_tracker, columns, server_display_name, thread_id); } - block.setColumns(std::move(columns)); - initProfileEventsBlockOutput(block); + MutableColumns logs_columns; + Block curr_block; + size_t rows = 0; - writeVarUInt(Protocol::Server::ProfileEvents, *out); - writeStringBinary("", *out); + bool from_queue = false; + for (; state.profile_queue->tryPop(curr_block); ++rows) + { + from_queue = true; + auto curr_columns = curr_block.getColumns(); + for (size_t j = 0; j < curr_columns.size(); ++j) + columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); + } - state.profile_events_block_out->write(block); - out->next(); + bool empty = true; + for (auto & column : columns) + empty = empty && column->empty(); + + if (!empty) + { + block.setColumns(std::move(columns)); + + initProfileEventsBlockOutput(block); + + writeVarUInt(Protocol::Server::ProfileEvents, *out); + writeStringBinary("", *out); + + state.profile_events_block_out->write(block); + out->next(); + LOG_DEBUG(log, "Sent ProfileEvents packet {} data from queue", (from_queue ? "with" : "without")); + } } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 9ff061e096b..b5d7d1f0776 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -48,6 +48,7 @@ struct QueryState InternalTextLogsQueuePtr logs_queue; std::unique_ptr logs_block_out; + InternalProfileEventsQueuePtr profile_queue; std::unique_ptr profile_events_block_out; /// From where to read data for INSERT. From 9071a7151ef7171c22c27570d95ddb2a018f12f6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Sep 2021 15:07:24 +0300 Subject: [PATCH 438/950] Fix communication & race conditions --- src/Common/ThreadStatus.cpp | 6 +++++ src/Server/TCPHandler.cpp | 46 ++++++++++++++++++++++++------------- 2 files changed, 36 insertions(+), 16 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index 4c49e9b1d0d..d521106a29b 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -142,6 +142,12 @@ ThreadStatus::~ThreadStatus() /// We've already allocated a little bit more than the limit and cannot track it in the thread memory tracker or its parent. } + if (thread_group) + { + std::lock_guard guard(thread_group->mutex); + thread_group->threads.erase(this); + } + #if !defined(ARCADIA_BUILD) /// It may cause segfault if query_context was destroyed, but was not detached auto query_context_ptr = query_context.lock(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f1014d611fd..581cba91356 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -835,6 +837,14 @@ namespace constexpr size_t NAME_COLUMN_INDEX = 4; constexpr size_t VALUE_COLUMN_INDEX = 5; + struct ProfileEventsSnapshot + { + UInt64 thread_id; + ProfileEvents::Counters counters; + CurrentMetrics::Metric metric; + time_t current_time; + }; + /* * Add records about provided non-zero ProfileEvents::Counters. */ @@ -873,12 +883,11 @@ namespace } void dumpMemoryTracker( - MemoryTracker * memoryTracker, + CurrentMetrics::Metric metric, MutableColumns & columns, String const & host_name, UInt64 thread_id) { - auto metric = memoryTracker->getMetric(); if (metric == CurrentMetrics::end()) return; time_t current_time = time(nullptr); @@ -923,20 +932,28 @@ void TCPHandler::sendProfileEvents() MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); - std::vector threads; + std::vector snapshots; { std::lock_guard guard(thread_group->mutex); - std::copy(thread_group->threads.begin(), thread_group->threads.end(), std::back_inserter(threads)); + for (auto * thread : thread_group->threads) + { + auto current_time = time(nullptr); + auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); + auto metric = thread->memory_tracker.getMetric(); + auto const thread_id = CurrentThread::get().thread_id; + snapshots.push_back(ProfileEventsSnapshot{thread_id, std::move(counters), metric, current_time}); + } } - for (auto * thread : threads) - { - auto const counters_snapshot = thread->performance_counters.getPartiallyAtomicSnapshot(); - auto current_time = time(nullptr); - auto * memory_tracker = &thread->memory_tracker; - auto const thread_id = CurrentThread::get().thread_id; - dumpProfileEvents(counters_snapshot, columns, server_display_name, current_time, thread_id); - dumpMemoryTracker(memory_tracker, columns, server_display_name, thread_id); + for (auto & snapshot : snapshots) + { + dumpProfileEvents( + snapshot.counters, + columns, + server_display_name, + snapshot.current_time, + snapshot.thread_id); + dumpMemoryTracker(snapshot.metric, columns, server_display_name, snapshot.thread_id); } MutableColumns logs_columns; @@ -952,10 +969,7 @@ void TCPHandler::sendProfileEvents() columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); } - bool empty = true; - for (auto & column : columns) - empty = empty && column->empty(); - + bool empty = columns[0]->empty(); if (!empty) { block.setColumns(std::move(columns)); From 15ac65aa33fa77f6fa7ff0d67d40db3043f4d634 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 14 Sep 2021 14:06:00 +0300 Subject: [PATCH 439/950] Add thread usage info on client side --- src/Client/ClientBase.cpp | 16 +++++++++++++--- src/Client/ClientBase.h | 2 +- src/Common/ProgressIndication.cpp | 9 +++++++++ src/Common/ProgressIndication.h | 8 ++++++++ 4 files changed, 31 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ee5f3580050..988f008fef7 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -9,6 +9,8 @@ #include #include #include +#include "Columns/ColumnsNumber.h" +#include "Core/Block.h" #include "Core/Protocol.h" #if !defined(ARCADIA_BUILD) @@ -613,7 +615,7 @@ bool ClientBase::receiveAndProcessPacket(ASTPtr parsed_query, bool cancelled) return false; case Protocol::Server::ProfileEvents: - onProfileEvents(); + onProfileEvents(packet.block); return true; default: @@ -656,8 +658,16 @@ void ClientBase::onEndOfStream() } -void ClientBase::onProfileEvents() -{} +void ClientBase::onProfileEvents(Block & block) +{ + if (block.rows() == 0) + return; + const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); + for (size_t i = 0; i < block.rows(); ++i) + { + progress_indication.addThreadIdToList(array_thread_id[i]); + } +} /// Flush all buffers. diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 0fa205a4d6e..070b676366c 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -114,7 +114,7 @@ private: void onReceiveExceptionFromServer(std::unique_ptr && e); void onProfileInfo(const BlockStreamProfileInfo & profile_info); void onEndOfStream(); - void onProfileEvents(); + void onProfileEvents(Block & block); void sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query); void sendDataFrom(ReadBuffer & buf, Block & sample, diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 0d65eaece86..02bb7d202d7 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -29,6 +29,7 @@ void ProgressIndication::resetProgress() show_progress_bar = false; written_progress_chars = 0; write_progress_on_update = false; + thread_ids.clear(); } void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool write_progress_on_update_) @@ -43,6 +44,11 @@ void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool }); } +void ProgressIndication::addThreadIdToList(UInt64 thread_id) +{ + thread_ids.insert(thread_id); +} + void ProgressIndication::writeFinalProgress() { if (progress.read_rows < 1000) @@ -57,6 +63,9 @@ void ProgressIndication::writeFinalProgress() << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; else std::cout << ". "; + size_t used_threads = getUsedThreadsCount(); + if (used_threads != 0) + std::cout << "\nUsed threads to process: " << used_threads << "."; } void ProgressIndication::writeProgress() diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 044d8cb1a89..ba7889c7326 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -1,7 +1,9 @@ #pragma once +#include #include #include +#include #include @@ -41,6 +43,10 @@ public: /// How much seconds passed since query execution start. double elapsedSeconds() const { return watch.elapsedSeconds(); } + void addThreadIdToList(UInt64 thread_id); + + size_t getUsedThreadsCount() const { return thread_ids.size(); } + private: /// This flag controls whether to show the progress bar. We start showing it after /// the query has been executing for 0.5 seconds, and is still less than half complete. @@ -58,6 +64,8 @@ private: Stopwatch watch; bool write_progress_on_update = false; + + std::unordered_set thread_ids; }; } From 4c6b3c40f2f3b854b85c5ff640ad9ee0f3bbe704 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 14 Sep 2021 16:24:57 +0300 Subject: [PATCH 440/950] Calculate approximate cores number used --- src/Client/ClientBase.cpp | 26 +++++++++++++++++++- src/Common/ProgressIndication.cpp | 41 ++++++++++++++++++++++++++++--- src/Common/ProgressIndication.h | 17 +++++++++++-- 3 files changed, 78 insertions(+), 6 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 988f008fef7..5e0e11d103e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -9,6 +9,7 @@ #include #include #include +#include "Columns/ColumnString.h" #include "Columns/ColumnsNumber.h" #include "Core/Block.h" #include "Core/Protocol.h" @@ -75,6 +76,12 @@ namespace ErrorCodes } +namespace ProfileEvents +{ + extern const Event UserTimeMicroseconds; + extern const Event SystemTimeMicroseconds; +} + namespace DB { @@ -663,9 +670,26 @@ void ClientBase::onProfileEvents(Block & block) if (block.rows() == 0) return; const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); + const auto & names = typeid_cast(*block.getByName("name").column); + const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); + + auto const * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); + auto const * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); + for (size_t i = 0; i < block.rows(); ++i) { - progress_indication.addThreadIdToList(array_thread_id[i]); + auto thread_id = array_thread_id[i]; + progress_indication.addThreadIdToList(thread_id); + auto event_name = names.getDataAt(i); + auto value = array_values[i]; + if (event_name == user_time_name) + { + progress_indication.updateThreadUserTime(thread_id, value); + } + else if (event_name == system_time_name) + { + progress_indication.updateThreadSystemTime(thread_id, value); + } } } diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 02bb7d202d7..ceb039b15f5 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -1,5 +1,8 @@ #include "ProgressIndication.h" +#include +#include #include +#include #include #include #include @@ -29,7 +32,7 @@ void ProgressIndication::resetProgress() show_progress_bar = false; written_progress_chars = 0; write_progress_on_update = false; - thread_ids.clear(); + thread_times.clear(); } void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool write_progress_on_update_) @@ -46,7 +49,28 @@ void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool void ProgressIndication::addThreadIdToList(UInt64 thread_id) { - thread_ids.insert(thread_id); + if (thread_times.contains(thread_id)) + return; + thread_times[thread_id] = {}; +} + +void ProgressIndication::updateThreadUserTime(UInt64 thread_id, UInt64 value) +{ + thread_times[thread_id].user_ms = value; +} + +void ProgressIndication::updateThreadSystemTime(UInt64 thread_id, UInt64 value) +{ + thread_times[thread_id].system_ms = value; +} + +UInt64 ProgressIndication::getAccumulatedThreadTime() const +{ + return std::accumulate(thread_times.cbegin(), thread_times.cend(), static_cast(0), + [](UInt64 acc, auto const & elem) + { + return acc + elem.second.user_ms + elem.second.system_ms; + }); } void ProgressIndication::writeFinalProgress() @@ -63,9 +87,20 @@ void ProgressIndication::writeFinalProgress() << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; else std::cout << ". "; + size_t used_threads = getUsedThreadsCount(); if (used_threads != 0) - std::cout << "\nUsed threads to process: " << used_threads << "."; + { + std::cout << "\nUsed threads to process: " << used_threads; + + auto elapsed_ms = watch.elapsedMicroseconds(); + auto accumulated_thread_times = getAccumulatedThreadTime(); + auto approximate_core_number = (accumulated_thread_times + elapsed_ms - 1) / elapsed_ms; + if (approximate_core_number != 0) + std::cout << " and cores: " << approximate_core_number << "."; + else + std::cout << "."; + } } void ProgressIndication::writeProgress() diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index ba7889c7326..7517853f74d 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -45,9 +45,16 @@ public: void addThreadIdToList(UInt64 thread_id); - size_t getUsedThreadsCount() const { return thread_ids.size(); } + void updateThreadUserTime(UInt64 thread_id, UInt64 value); + + void updateThreadSystemTime(UInt64 thread_id, UInt64 value); private: + + size_t getUsedThreadsCount() const { return thread_times.size(); } + + UInt64 getAccumulatedThreadTime() const; + /// This flag controls whether to show the progress bar. We start showing it after /// the query has been executing for 0.5 seconds, and is still less than half complete. bool show_progress_bar = false; @@ -65,7 +72,13 @@ private: bool write_progress_on_update = false; - std::unordered_set thread_ids; + struct ThreadTime + { + UInt64 user_ms = 0; + UInt64 system_ms = 0; + }; + + std::unordered_map thread_times; }; } From 7e3caf96bec1afefe26eff8639515ad869e635de Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 15 Sep 2021 18:45:43 +0300 Subject: [PATCH 441/950] Fix cores approximation --- src/Client/ClientBase.cpp | 9 +++-- src/Common/ProgressIndication.cpp | 56 +++++++++++++++++++++++-------- src/Common/ProgressIndication.h | 16 +++++---- src/Server/TCPHandler.cpp | 11 +++++- 4 files changed, 68 insertions(+), 24 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5e0e11d103e..6a05ebd7c1b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -671,6 +671,7 @@ void ClientBase::onProfileEvents(Block & block) return; const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & names = typeid_cast(*block.getByName("name").column); + const auto & host_names = typeid_cast(*block.getByName("host_name").column); const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); auto const * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); @@ -679,16 +680,18 @@ void ClientBase::onProfileEvents(Block & block) for (size_t i = 0; i < block.rows(); ++i) { auto thread_id = array_thread_id[i]; - progress_indication.addThreadIdToList(thread_id); + auto host_name = host_names.getDataAt(i).toString(); + if (thread_id != 0) + progress_indication.addThreadIdToList(host_name, thread_id); auto event_name = names.getDataAt(i); auto value = array_values[i]; if (event_name == user_time_name) { - progress_indication.updateThreadUserTime(thread_id, value); + progress_indication.updateThreadUserTime(host_name, thread_id, value); } else if (event_name == system_time_name) { - progress_indication.updateThreadSystemTime(thread_id, value); + progress_indication.updateThreadSystemTime(host_name, thread_id, value); } } } diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index ceb039b15f5..b06df1bba15 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -1,4 +1,5 @@ #include "ProgressIndication.h" +#include #include #include #include @@ -8,6 +9,11 @@ #include +namespace +{ + constexpr UInt64 ZERO = 0; +} + namespace DB { @@ -47,29 +53,53 @@ void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool }); } -void ProgressIndication::addThreadIdToList(UInt64 thread_id) +void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id) { - if (thread_times.contains(thread_id)) + auto & thread_to_times = thread_times[host]; + if (thread_to_times.contains(thread_id)) return; - thread_times[thread_id] = {}; + thread_to_times[thread_id] = {}; } -void ProgressIndication::updateThreadUserTime(UInt64 thread_id, UInt64 value) +void ProgressIndication::updateThreadUserTime(String const & host, UInt64 thread_id, UInt64 value) { - thread_times[thread_id].user_ms = value; + thread_times[host][thread_id].user_ms = value; } -void ProgressIndication::updateThreadSystemTime(UInt64 thread_id, UInt64 value) +void ProgressIndication::updateThreadSystemTime(String const & host, UInt64 thread_id, UInt64 value) { - thread_times[thread_id].system_ms = value; + thread_times[host][thread_id].system_ms = value; } -UInt64 ProgressIndication::getAccumulatedThreadTime() const +size_t ProgressIndication::getUsedThreadsCount() const { - return std::accumulate(thread_times.cbegin(), thread_times.cend(), static_cast(0), - [](UInt64 acc, auto const & elem) + return std::accumulate(thread_times.cbegin(), thread_times.cend(), 0, + [] (size_t acc, auto const & threads) { - return acc + elem.second.user_ms + elem.second.system_ms; + return acc + threads.second.size(); + }); +} + +UInt64 ProgressIndication::getApproximateCoresNumber() const +{ + return std::accumulate(thread_times.cbegin(), thread_times.cend(), ZERO, + [](UInt64 acc, auto const & threads) + { + auto total_time = std::accumulate(threads.second.cbegin(), threads.second.cend(), ZERO, + [] (UInt64 temp, auto const & elem) + { + if (elem.first == 0) + return temp; + return temp + elem.second.user_ms + elem.second.system_ms; + }); + // Zero thread_id represents thread group which execute query + // (including thread of TCPHandler). + auto const & accumulated_time = threads.second.find(ZERO)->second; + // Performance events of TCPHandler thread are not transmitted, but + // we can calculate it's working time which shows how long the query + // is being processed. + auto io_time = accumulated_time.user_ms + accumulated_time.system_ms - total_time; + return acc + (total_time + io_time - 1) / io_time; }); } @@ -93,9 +123,7 @@ void ProgressIndication::writeFinalProgress() { std::cout << "\nUsed threads to process: " << used_threads; - auto elapsed_ms = watch.elapsedMicroseconds(); - auto accumulated_thread_times = getAccumulatedThreadTime(); - auto approximate_core_number = (accumulated_thread_times + elapsed_ms - 1) / elapsed_ms; + auto approximate_core_number = getApproximateCoresNumber(); if (approximate_core_number != 0) std::cout << " and cores: " << approximate_core_number << "."; else diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 7517853f74d..f1d7d214f4f 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -43,17 +44,17 @@ public: /// How much seconds passed since query execution start. double elapsedSeconds() const { return watch.elapsedSeconds(); } - void addThreadIdToList(UInt64 thread_id); + void addThreadIdToList(String const & host, UInt64 thread_id); - void updateThreadUserTime(UInt64 thread_id, UInt64 value); + void updateThreadUserTime(String const & host, UInt64 thread_id, UInt64 value); - void updateThreadSystemTime(UInt64 thread_id, UInt64 value); + void updateThreadSystemTime(String const & host, UInt64 thread_id, UInt64 value); private: - size_t getUsedThreadsCount() const { return thread_times.size(); } + size_t getUsedThreadsCount() const; - UInt64 getAccumulatedThreadTime() const; + UInt64 getApproximateCoresNumber() const; /// This flag controls whether to show the progress bar. We start showing it after /// the query has been executing for 0.5 seconds, and is still less than half complete. @@ -78,7 +79,10 @@ private: UInt64 system_ms = 0; }; - std::unordered_map thread_times; + using ThreadIdToTimeMap = std::unordered_map; + using HostToThreadTimesMap = std::unordered_map; + + HostToThreadTimesMap thread_times; }; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 581cba91356..c24bf599527 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -932,19 +932,28 @@ void TCPHandler::sendProfileEvents() MutableColumns columns = block.mutateColumns(); auto thread_group = CurrentThread::getGroup(); + auto const current_thread_id = CurrentThread::get().thread_id; std::vector snapshots; + ProfileEventsSnapshot group_snapshot; { std::lock_guard guard(thread_group->mutex); for (auto * thread : thread_group->threads) { + auto const thread_id = thread->thread_id; + if (thread_id == current_thread_id) + continue; auto current_time = time(nullptr); auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); auto metric = thread->memory_tracker.getMetric(); - auto const thread_id = CurrentThread::get().thread_id; snapshots.push_back(ProfileEventsSnapshot{thread_id, std::move(counters), metric, current_time}); } + group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); + group_snapshot.metric = thread_group->memory_tracker.getMetric(); + group_snapshot.current_time = time(nullptr); } + dumpProfileEvents(group_snapshot.counters, columns, server_display_name, group_snapshot.current_time, 0); + dumpMemoryTracker(group_snapshot.metric, columns, server_display_name, 0); for (auto & snapshot : snapshots) { dumpProfileEvents( From 1d2e2d73057ebd9dd791cf9e4f98107da2fd5e88 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 15 Sep 2021 23:35:04 +0300 Subject: [PATCH 442/950] cleanup --- src/DataStreams/RemoteQueryExecutor.cpp | 1 - src/Server/TCPHandler.cpp | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 51c5c2edc57..aa316e54e6f 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -396,7 +396,6 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) case Protocol::Server::ProfileEvents: /// Pass profile events from remote server to client { - LOG_DEBUG(log, "RemoteQueryExecutor received ProfileEvents"); auto profile_queue = CurrentThread::getInternalProfileEventsQueue(); profile_queue->emplace(std::move(packet.block)); break; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c24bf599527..04b85125d66 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -675,10 +675,10 @@ void TCPHandler::processOrdinaryQueryWithProcessors() /// Some time passed and there is a progress. after_send_progress.restart(); sendProgress(); + sendProfileEvents(); } sendLogs(); - sendProfileEvents(); if (block) { @@ -701,7 +701,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors() sendProfileInfo(executor.getProfileInfo()); sendProgress(); sendLogs(); - // sendProfileEvents(); + sendProfileEvents(); } if (state.is_connection_closed) From 73df6190df357ecf219be454d116321a89c3319e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 17 Sep 2021 18:00:13 +0300 Subject: [PATCH 443/950] Cleanup code --- src/Client/ClientBase.cpp | 6 ++- src/Client/Connection.cpp | 1 - src/Common/ProgressIndication.cpp | 69 ++++++++++++++++++++----------- src/Common/ProgressIndication.h | 25 +++++------ src/Server/TCPHandler.cpp | 12 +++--- 5 files changed, 67 insertions(+), 46 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6a05ebd7c1b..b3148525f8c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -677,6 +677,7 @@ void ClientBase::onProfileEvents(Block & block) auto const * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); auto const * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); + HostToThreadTimesMap thread_times; for (size_t i = 0; i < block.rows(); ++i) { auto thread_id = array_thread_id[i]; @@ -687,13 +688,14 @@ void ClientBase::onProfileEvents(Block & block) auto value = array_values[i]; if (event_name == user_time_name) { - progress_indication.updateThreadUserTime(host_name, thread_id, value); + thread_times[host_name][thread_id].user_ms = value; } else if (event_name == system_time_name) { - progress_indication.updateThreadSystemTime(host_name, thread_id, value); + thread_times[host_name][thread_id].system_ms = value; } } + progress_indication.updateThreadTimes(thread_times); } diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index c6badf96bf9..40f74bcf9a7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -873,7 +873,6 @@ Packet Connection::receivePacket() return res; case Protocol::Server::ProfileEvents: - LOG_DEBUG(log_wrapper.get(), "Connection received ProfileEvents"); res.block = receiveProfileEvents(); return res; diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index b06df1bba15..9a87a86c76b 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -12,6 +12,28 @@ namespace { constexpr UInt64 ZERO = 0; + + UInt64 calculateNewCoresNumber(DB::ThreadIdToTimeMap const & prev, DB::ThreadIdToTimeMap const& next) + { + if (next.find(ZERO) == next.end()) + return ZERO; + auto accumulated = std::accumulate(next.cbegin(), next.cend(), ZERO, + [&prev](UInt64 acc, auto const & elem) + { + if (elem.first == ZERO) + return acc; + auto thread_time = elem.second.time(); + auto it = prev.find(elem.first); + if (it != prev.end()) + thread_time -= it->second.time(); + return acc + thread_time; + }); + + auto elapsed = next.at(ZERO).time() - (prev.contains(ZERO) ? prev.at(ZERO).time() : ZERO); + if (elapsed == ZERO) + return ZERO; + return (accumulated + elapsed - 1) / elapsed; + } } namespace DB @@ -38,6 +60,7 @@ void ProgressIndication::resetProgress() show_progress_bar = false; written_progress_chars = 0; write_progress_on_update = false; + host_active_cores.clear(); thread_times.clear(); } @@ -61,14 +84,15 @@ void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id thread_to_times[thread_id] = {}; } -void ProgressIndication::updateThreadUserTime(String const & host, UInt64 thread_id, UInt64 value) +void ProgressIndication::updateThreadTimes(HostToThreadTimesMap & new_thread_times) { - thread_times[host][thread_id].user_ms = value; -} - -void ProgressIndication::updateThreadSystemTime(String const & host, UInt64 thread_id, UInt64 value) -{ - thread_times[host][thread_id].system_ms = value; + for (auto & new_host_map : new_thread_times) + { + auto & host_map = thread_times[new_host_map.first]; + auto new_cores = calculateNewCoresNumber(host_map, new_host_map.second); + host_active_cores[new_host_map.first] = new_cores; + host_map = std::move(new_host_map.second); + } } size_t ProgressIndication::getUsedThreadsCount() const @@ -82,24 +106,10 @@ size_t ProgressIndication::getUsedThreadsCount() const UInt64 ProgressIndication::getApproximateCoresNumber() const { - return std::accumulate(thread_times.cbegin(), thread_times.cend(), ZERO, - [](UInt64 acc, auto const & threads) + return std::accumulate(host_active_cores.cbegin(), host_active_cores.cend(), ZERO, + [](UInt64 acc, auto const & elem) { - auto total_time = std::accumulate(threads.second.cbegin(), threads.second.cend(), ZERO, - [] (UInt64 temp, auto const & elem) - { - if (elem.first == 0) - return temp; - return temp + elem.second.user_ms + elem.second.system_ms; - }); - // Zero thread_id represents thread group which execute query - // (including thread of TCPHandler). - auto const & accumulated_time = threads.second.find(ZERO)->second; - // Performance events of TCPHandler thread are not transmitted, but - // we can calculate it's working time which shows how long the query - // is being processed. - auto io_time = accumulated_time.user_ms + accumulated_time.system_ms - total_time; - return acc + (total_time + io_time - 1) / io_time; + return acc + elem.second; }); } @@ -220,6 +230,17 @@ void ProgressIndication::writeProgress() message << ' ' << (99 * current_count / max_count) << '%'; } + // If approximate cores number is known, display it. + auto cores_number = getApproximateCoresNumber(); + if (cores_number != 0) + { + // Calculated cores number may be not accurate + // so it's better to print min(threads, cores). + auto threads_number = getUsedThreadsCount(); + message << " Running " << threads_number << " threads on " + << std::min(cores_number, threads_number) << " cores."; + } + message << CLEAR_TO_END_OF_LINE; ++increment; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index f1d7d214f4f..4a98b5e849b 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -14,6 +14,17 @@ namespace DB { +struct ThreadTime +{ + UInt64 time() const noexcept { return user_ms + system_ms; } + + UInt64 user_ms = 0; + UInt64 system_ms = 0; +}; + +using ThreadIdToTimeMap = std::unordered_map; +using HostToThreadTimesMap = std::unordered_map; + class ProgressIndication { public: @@ -46,9 +57,7 @@ public: void addThreadIdToList(String const & host, UInt64 thread_id); - void updateThreadUserTime(String const & host, UInt64 thread_id, UInt64 value); - - void updateThreadSystemTime(String const & host, UInt64 thread_id, UInt64 value); + void updateThreadTimes(HostToThreadTimesMap & new_thread_times); private: @@ -73,15 +82,7 @@ private: bool write_progress_on_update = false; - struct ThreadTime - { - UInt64 user_ms = 0; - UInt64 system_ms = 0; - }; - - using ThreadIdToTimeMap = std::unordered_map; - using HostToThreadTimesMap = std::unordered_map; - + std::unordered_map host_active_cores; HostToThreadTimesMap thread_times; }; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 04b85125d66..9bca044617a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -947,13 +947,12 @@ void TCPHandler::sendProfileEvents() auto metric = thread->memory_tracker.getMetric(); snapshots.push_back(ProfileEventsSnapshot{thread_id, std::move(counters), metric, current_time}); } - group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); - group_snapshot.metric = thread_group->memory_tracker.getMetric(); + group_snapshot.current_time = time(nullptr); + group_snapshot.metric = thread_group->memory_tracker.getMetric(); + group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); } - dumpProfileEvents(group_snapshot.counters, columns, server_display_name, group_snapshot.current_time, 0); - dumpMemoryTracker(group_snapshot.metric, columns, server_display_name, 0); for (auto & snapshot : snapshots) { dumpProfileEvents( @@ -964,15 +963,15 @@ void TCPHandler::sendProfileEvents() snapshot.thread_id); dumpMemoryTracker(snapshot.metric, columns, server_display_name, snapshot.thread_id); } + dumpProfileEvents(group_snapshot.counters, columns, server_display_name, group_snapshot.current_time, 0); + dumpMemoryTracker(group_snapshot.metric, columns, server_display_name, 0); MutableColumns logs_columns; Block curr_block; size_t rows = 0; - bool from_queue = false; for (; state.profile_queue->tryPop(curr_block); ++rows) { - from_queue = true; auto curr_columns = curr_block.getColumns(); for (size_t j = 0; j < curr_columns.size(); ++j) columns[j]->insertRangeFrom(*curr_columns[j], 0, curr_columns[j]->size()); @@ -990,7 +989,6 @@ void TCPHandler::sendProfileEvents() state.profile_events_block_out->write(block); out->next(); - LOG_DEBUG(log, "Sent ProfileEvents packet {} data from queue", (from_queue ? "with" : "without")); } } From 9f9af28b5ec5eb58f73c16f4d2737c3f284d163f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 17 Sep 2021 19:47:54 +0300 Subject: [PATCH 444/950] Output memory usage with progress --- src/Client/ClientBase.cpp | 6 ++- src/Common/MemoryTracker.h | 3 ++ src/Common/ProgressIndication.cpp | 45 ++++++++++------- src/Common/ProgressIndication.h | 15 +++--- src/Server/TCPHandler.cpp | 82 ++++++++++++++++++------------- 5 files changed, 91 insertions(+), 60 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b3148525f8c..1e104292e06 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -694,8 +694,12 @@ void ClientBase::onProfileEvents(Block & block) { thread_times[host_name][thread_id].system_ms = value; } + else if (event_name == MemoryTracker::USAGE_EVENT_NAME) + { + thread_times[host_name][thread_id].memory_usage = value; + } } - progress_indication.updateThreadTimes(thread_times); + progress_indication.updateThreadEventData(thread_times); } diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 7da70db0876..ce0eef52e17 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -64,6 +64,9 @@ private: void setOrRaiseProfilerLimit(Int64 value); public: + + static constexpr auto USAGE_EVENT_NAME = "MemoryTrackerUsage"; + explicit MemoryTracker(VariableContext level_ = VariableContext::Thread); explicit MemoryTracker(MemoryTracker * parent_, VariableContext level_ = VariableContext::Thread); diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 9a87a86c76b..189af2e9972 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -61,7 +61,7 @@ void ProgressIndication::resetProgress() written_progress_chars = 0; write_progress_on_update = false; host_active_cores.clear(); - thread_times.clear(); + thread_data.clear(); } void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool write_progress_on_update_) @@ -78,17 +78,17 @@ void ProgressIndication::setFileProgressCallback(ContextMutablePtr context, bool void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id) { - auto & thread_to_times = thread_times[host]; + auto & thread_to_times = thread_data[host]; if (thread_to_times.contains(thread_id)) return; thread_to_times[thread_id] = {}; } -void ProgressIndication::updateThreadTimes(HostToThreadTimesMap & new_thread_times) +void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data) { - for (auto & new_host_map : new_thread_times) + for (auto & new_host_map : new_thread_data) { - auto & host_map = thread_times[new_host_map.first]; + auto & host_map = thread_data[new_host_map.first]; auto new_cores = calculateNewCoresNumber(host_map, new_host_map.second); host_active_cores[new_host_map.first] = new_cores; host_map = std::move(new_host_map.second); @@ -97,7 +97,7 @@ void ProgressIndication::updateThreadTimes(HostToThreadTimesMap & new_thread_tim size_t ProgressIndication::getUsedThreadsCount() const { - return std::accumulate(thread_times.cbegin(), thread_times.cend(), 0, + return std::accumulate(thread_data.cbegin(), thread_data.cend(), 0, [] (size_t acc, auto const & threads) { return acc + threads.second.size(); @@ -113,6 +113,19 @@ UInt64 ProgressIndication::getApproximateCoresNumber() const }); } +UInt64 ProgressIndication::getMemoryUsage() const +{ + return std::accumulate(thread_data.cbegin(), thread_data.cend(), ZERO, + [](UInt64 acc, auto const & host_data) + { + return acc + std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, + [](UInt64 memory, auto const & data) + { + return memory + data.second.memory_usage; + }); + }); +} + void ProgressIndication::writeFinalProgress() { if (progress.read_rows < 1000) @@ -127,18 +140,6 @@ void ProgressIndication::writeFinalProgress() << formatReadableSizeWithDecimalSuffix(progress.read_bytes * 1000000000.0 / elapsed_ns) << "/s.)"; else std::cout << ". "; - - size_t used_threads = getUsedThreadsCount(); - if (used_threads != 0) - { - std::cout << "\nUsed threads to process: " << used_threads; - - auto approximate_core_number = getApproximateCoresNumber(); - if (approximate_core_number != 0) - std::cout << " and cores: " << approximate_core_number << "."; - else - std::cout << "."; - } } void ProgressIndication::writeProgress() @@ -238,7 +239,13 @@ void ProgressIndication::writeProgress() // so it's better to print min(threads, cores). auto threads_number = getUsedThreadsCount(); message << " Running " << threads_number << " threads on " - << std::min(cores_number, threads_number) << " cores."; + << std::min(cores_number, threads_number) << " cores"; + + auto memory_usage = getMemoryUsage(); + if (memory_usage != 0) + message << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; + else + message << "."; } message << CLEAR_TO_END_OF_LINE; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 4a98b5e849b..3d9bbc7f3ff 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -14,15 +14,16 @@ namespace DB { -struct ThreadTime +struct ThreadEventData { UInt64 time() const noexcept { return user_ms + system_ms; } - UInt64 user_ms = 0; - UInt64 system_ms = 0; + UInt64 user_ms = 0; + UInt64 system_ms = 0; + UInt64 memory_usage = 0; }; -using ThreadIdToTimeMap = std::unordered_map; +using ThreadIdToTimeMap = std::unordered_map; using HostToThreadTimesMap = std::unordered_map; class ProgressIndication @@ -57,7 +58,7 @@ public: void addThreadIdToList(String const & host, UInt64 thread_id); - void updateThreadTimes(HostToThreadTimesMap & new_thread_times); + void updateThreadEventData(HostToThreadTimesMap & new_thread_data); private: @@ -65,6 +66,8 @@ private: UInt64 getApproximateCoresNumber() const; + UInt64 getMemoryUsage() const; + /// This flag controls whether to show the progress bar. We start showing it after /// the query has been executing for 0.5 seconds, and is still less than half complete. bool show_progress_bar = false; @@ -83,7 +86,7 @@ private: bool write_progress_on_update = false; std::unordered_map host_active_cores; - HostToThreadTimesMap thread_times; + HostToThreadTimesMap thread_data; }; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9bca044617a..45a65f990df 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -842,6 +842,7 @@ namespace UInt64 thread_id; ProfileEvents::Counters counters; CurrentMetrics::Metric metric; + Int64 memory_usage; time_t current_time; }; @@ -849,18 +850,16 @@ namespace * Add records about provided non-zero ProfileEvents::Counters. */ void dumpProfileEvents( - ProfileEvents::Counters const & snapshot, + ProfileEventsSnapshot const & snapshot, MutableColumns & columns, - String const & host_name, - time_t current_time, - UInt64 thread_id) + String const & host_name) { size_t rows = 0; auto & name_column = columns[NAME_COLUMN_INDEX]; auto & value_column = columns[VALUE_COLUMN_INDEX]; for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) { - UInt64 value = snapshot[event].load(std::memory_order_relaxed); + UInt64 value = snapshot.counters[event].load(std::memory_order_relaxed); if (value == 0) continue; @@ -876,32 +875,43 @@ namespace { size_t i = 0; columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(current_time)); - columns[i++]->insert(UInt64{thread_id}); + columns[i++]->insert(UInt64(snapshot.current_time)); + columns[i++]->insert(UInt64{snapshot.thread_id}); columns[i++]->insert(ProfileEventTypes::INCREMENT); } } void dumpMemoryTracker( - CurrentMetrics::Metric metric, + ProfileEventsSnapshot const & snapshot, MutableColumns & columns, - String const & host_name, - UInt64 thread_id) + String const & host_name) { - if (metric == CurrentMetrics::end()) - return; - time_t current_time = time(nullptr); + { + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(snapshot.current_time)); + columns[i++]->insert(UInt64{snapshot.thread_id}); + columns[i++]->insert(ProfileEventTypes::GAUGE); - size_t i = 0; - columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(current_time)); - columns[i++]->insert(UInt64{thread_id}); - columns[i++]->insert(ProfileEventTypes::GAUGE); + columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); + columns[i++]->insert(snapshot.memory_usage); + } - auto const * metric_name = CurrentMetrics::getName(metric); - columns[i++]->insertData(metric_name, strlen(metric_name)); - auto metric_value = CurrentMetrics::get(metric); - columns[i++]->insert(metric_value); + if (snapshot.metric != CurrentMetrics::end()) + { + time_t current_time = time(nullptr); + + size_t i = 0; + columns[i++]->insertData(host_name.data(), host_name.size()); + columns[i++]->insert(UInt64(current_time)); + columns[i++]->insert(UInt64{snapshot.thread_id}); + columns[i++]->insert(ProfileEventTypes::GAUGE); + + auto const * metric_name = CurrentMetrics::getName(snapshot.metric); + columns[i++]->insertData(metric_name, strlen(metric_name)); + auto metric_value = CurrentMetrics::get(snapshot.metric); + columns[i++]->insert(metric_value); + } } } @@ -945,26 +955,30 @@ void TCPHandler::sendProfileEvents() auto current_time = time(nullptr); auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); auto metric = thread->memory_tracker.getMetric(); - snapshots.push_back(ProfileEventsSnapshot{thread_id, std::move(counters), metric, current_time}); + auto memory_usage = thread->memory_tracker.get(); + snapshots.push_back(ProfileEventsSnapshot{ + thread_id, + std::move(counters), + metric, + memory_usage, + current_time + }); } + group_snapshot.thread_id = 0; group_snapshot.current_time = time(nullptr); - group_snapshot.metric = thread_group->memory_tracker.getMetric(); - group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); + group_snapshot.metric = thread_group->memory_tracker.getMetric(); + group_snapshot.memory_usage = thread_group->memory_tracker.get(); + group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); } for (auto & snapshot : snapshots) { - dumpProfileEvents( - snapshot.counters, - columns, - server_display_name, - snapshot.current_time, - snapshot.thread_id); - dumpMemoryTracker(snapshot.metric, columns, server_display_name, snapshot.thread_id); + dumpProfileEvents(snapshot, columns, server_display_name); + dumpMemoryTracker(snapshot, columns, server_display_name); } - dumpProfileEvents(group_snapshot.counters, columns, server_display_name, group_snapshot.current_time, 0); - dumpMemoryTracker(group_snapshot.metric, columns, server_display_name, 0); + dumpProfileEvents(group_snapshot, columns, server_display_name); + dumpMemoryTracker(group_snapshot, columns, server_display_name); MutableColumns logs_columns; Block curr_block; From 0bdabf46f281a14d1138e2bd0956f10e746bc436 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 20 Sep 2021 18:59:27 +0300 Subject: [PATCH 445/950] Send ProfileEvents only to supported clients --- src/Core/ProtocolDefines.h | 4 +++- src/Server/TCPHandler.cpp | 3 +++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 37875305e75..6c62b969ff9 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -36,6 +36,8 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_DISTRIBUTED_DEPTH 54448 +#define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS 54450 + /// Version of ClickHouse TCP protocol. /// /// Should be incremented manually on protocol changes. @@ -43,6 +45,6 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54449 +#define DBMS_TCP_PROTOCOL_VERSION 54450 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 45a65f990df..c036715aea7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -918,6 +918,9 @@ namespace void TCPHandler::sendProfileEvents() { + if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS) + return; + auto profile_event_type = std::make_shared( DataTypeEnum8::Values { From c1f3e7e0bb8f5073c056c635cf101bdae4c7a941 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Sep 2021 00:47:48 +0300 Subject: [PATCH 446/950] Fix build --- src/Common/ProgressIndication.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 189af2e9972..0fe40b306cb 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -237,7 +237,7 @@ void ProgressIndication::writeProgress() { // Calculated cores number may be not accurate // so it's better to print min(threads, cores). - auto threads_number = getUsedThreadsCount(); + UInt64 threads_number = getUsedThreadsCount(); message << " Running " << threads_number << " threads on " << std::min(cores_number, threads_number) << " cores"; From 3a0764634e0be034c7e7ae93a4ec44a76d0e2b64 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Sep 2021 00:52:01 +0300 Subject: [PATCH 447/950] Update InternalProfileEventsQueue usage --- src/DataStreams/RemoteQueryExecutor.cpp | 6 ++---- src/Server/TCPHandler.cpp | 7 +++++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index aa316e54e6f..08d3db748b7 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -395,11 +395,9 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) case Protocol::Server::ProfileEvents: /// Pass profile events from remote server to client - { - auto profile_queue = CurrentThread::getInternalProfileEventsQueue(); + if (auto profile_queue = CurrentThread::getInternalProfileEventsQueue()) profile_queue->emplace(std::move(packet.block)); - break; - } + break; default: got_unknown_packet_from_replica = true; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c036715aea7..c25cb1dddfc 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -246,8 +246,11 @@ void TCPHandler::runImpl() sendLogs(); }); } - state.profile_queue = std::make_shared(std::numeric_limits::max()); - CurrentThread::attachInternalProfileEventsQueue(state.profile_queue); + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS) + { + state.profile_queue = std::make_shared(std::numeric_limits::max()); + CurrentThread::attachInternalProfileEventsQueue(state.profile_queue); + } query_context->setExternalTablesInitializer([this] (ContextPtr context) { From 9590c97a19f2330983e9ccbba5ba5fb7e5f16076 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 21 Sep 2021 15:29:04 +0300 Subject: [PATCH 448/950] Create ProfileEventsQueue in GRPCServer --- src/Server/GRPCServer.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index fc712916372..d0e054677c0 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1,4 +1,6 @@ #include "GRPCServer.h" +#include +#include #if USE_GRPC #include @@ -622,6 +624,7 @@ namespace BlockIO io; Progress progress; InternalTextLogsQueuePtr logs_queue; + InternalProfileEventsQueuePtr profile_queue; GRPCQueryInfo query_info; /// We reuse the same messages multiple times. GRPCResult result; @@ -773,6 +776,8 @@ namespace CurrentThread::attachInternalTextLogsQueue(logs_queue, client_logs_level); CurrentThread::setFatalErrorCallback([this]{ onFatalError(); }); } + profile_queue = std::make_shared(std::numeric_limits::max()); + CurrentThread::attachInternalProfileEventsQueue(profile_queue); /// Set the current database if specified. if (!query_info.database().empty()) From 7c3192735adda0ec7067dc4cfef92de47b9af938 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 6 Oct 2021 16:52:59 +0300 Subject: [PATCH 449/950] Reset profile events stream in Connection::sendQuery --- src/Client/Connection.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 40f74bcf9a7..1531e6c1e91 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -537,6 +537,7 @@ void Connection::sendQuery( maybe_compressed_out.reset(); block_in.reset(); block_logs_in.reset(); + block_profile_events_in.reset(); block_out.reset(); /// Send empty block which means end of data. From bfdd34c13d24466e96e06b185690960d07d33bb4 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Oct 2021 16:55:08 +0300 Subject: [PATCH 450/950] code cleanup --- src/Client/ClientBase.cpp | 9 +++++---- src/Server/GRPCServer.cpp | 11 ----------- src/Server/TCPHandler.cpp | 21 +-------------------- 3 files changed, 6 insertions(+), 35 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 1e104292e06..baf082a3541 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -667,18 +667,19 @@ void ClientBase::onEndOfStream() void ClientBase::onProfileEvents(Block & block) { - if (block.rows() == 0) + const auto rows = block.rows(); + if (rows == 0) return; const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & names = typeid_cast(*block.getByName("name").column); const auto & host_names = typeid_cast(*block.getByName("host_name").column); const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); - auto const * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); - auto const * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); + const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); + const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); HostToThreadTimesMap thread_times; - for (size_t i = 0; i < block.rows(); ++i) + for (size_t i = 0; i < rows; ++i) { auto thread_id = array_thread_id[i]; auto host_name = host_names.getDataAt(i).toString(); diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index d0e054677c0..3b01f3aedde 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -598,7 +598,6 @@ namespace void addExtremesToResult(const Block & extremes); void addProfileInfoToResult(const BlockStreamProfileInfo & info); void addLogsToResult(); - void addProfileEventsToResult(); void sendResult(); void throwIfFailedToSendResult(); void sendException(const Exception & exception); @@ -624,7 +623,6 @@ namespace BlockIO io; Progress progress; InternalTextLogsQueuePtr logs_queue; - InternalProfileEventsQueuePtr profile_queue; GRPCQueryInfo query_info; /// We reuse the same messages multiple times. GRPCResult result; @@ -776,8 +774,6 @@ namespace CurrentThread::attachInternalTextLogsQueue(logs_queue, client_logs_level); CurrentThread::setFatalErrorCallback([this]{ onFatalError(); }); } - profile_queue = std::make_shared(std::numeric_limits::max()); - CurrentThread::attachInternalProfileEventsQueue(profile_queue); /// Set the current database if specified. if (!query_info.database().empty()) @@ -1129,7 +1125,6 @@ namespace if (after_send_progress.elapsedMicroseconds() >= interactive_delay) { addProgressToResult(); - addProfileEventsToResult(); after_send_progress.restart(); } @@ -1181,7 +1176,6 @@ namespace finalize = true; io.onFinish(); addProgressToResult(); - addProfileEventsToResult(); query_scope->logPeakMemoryUsage(); addLogsToResult(); sendResult(); @@ -1445,11 +1439,6 @@ namespace } } - void Call::addProfileEventsToResult() - { - - } - void Call::sendResult() { /// gRPC doesn't allow to write anything to a finished responder. diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index c25cb1dddfc..2401b8614fa 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -844,7 +844,6 @@ namespace { UInt64 thread_id; ProfileEvents::Counters counters; - CurrentMetrics::Metric metric; Int64 memory_usage; time_t current_time; }; @@ -899,22 +898,6 @@ namespace columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); columns[i++]->insert(snapshot.memory_usage); } - - if (snapshot.metric != CurrentMetrics::end()) - { - time_t current_time = time(nullptr); - - size_t i = 0; - columns[i++]->insertData(host_name.data(), host_name.size()); - columns[i++]->insert(UInt64(current_time)); - columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEventTypes::GAUGE); - - auto const * metric_name = CurrentMetrics::getName(snapshot.metric); - columns[i++]->insertData(metric_name, strlen(metric_name)); - auto metric_value = CurrentMetrics::get(snapshot.metric); - columns[i++]->insert(metric_value); - } } } @@ -953,6 +936,7 @@ void TCPHandler::sendProfileEvents() ProfileEventsSnapshot group_snapshot; { std::lock_guard guard(thread_group->mutex); + snapshots.reserve(thread_group->threads.size()); for (auto * thread : thread_group->threads) { auto const thread_id = thread->thread_id; @@ -960,12 +944,10 @@ void TCPHandler::sendProfileEvents() continue; auto current_time = time(nullptr); auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); - auto metric = thread->memory_tracker.getMetric(); auto memory_usage = thread->memory_tracker.get(); snapshots.push_back(ProfileEventsSnapshot{ thread_id, std::move(counters), - metric, memory_usage, current_time }); @@ -973,7 +955,6 @@ void TCPHandler::sendProfileEvents() group_snapshot.thread_id = 0; group_snapshot.current_time = time(nullptr); - group_snapshot.metric = thread_group->memory_tracker.getMetric(); group_snapshot.memory_usage = thread_group->memory_tracker.get(); group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); } From e6c088fe056b919374ec4bd53c10b2c3ef9794b0 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 11 Oct 2021 13:03:00 +0300 Subject: [PATCH 451/950] Mark join_engine_deadlock as long test --- ...adlock.reference => 02033_join_engine_deadlock_long.reference} | 0 ...join_engine_deadlock.sh => 02033_join_engine_deadlock_long.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02033_join_engine_deadlock.reference => 02033_join_engine_deadlock_long.reference} (100%) rename tests/queries/0_stateless/{02033_join_engine_deadlock.sh => 02033_join_engine_deadlock_long.sh} (100%) diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.reference b/tests/queries/0_stateless/02033_join_engine_deadlock_long.reference similarity index 100% rename from tests/queries/0_stateless/02033_join_engine_deadlock.reference rename to tests/queries/0_stateless/02033_join_engine_deadlock_long.reference diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.sh b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh similarity index 100% rename from tests/queries/0_stateless/02033_join_engine_deadlock.sh rename to tests/queries/0_stateless/02033_join_engine_deadlock_long.sh From d301e4065190b4c590b2469112a2789959324168 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Oct 2021 18:31:09 +0300 Subject: [PATCH 452/950] Cache DNS in Poco handler for S3 if no proxy specified --- src/IO/S3/PocoHTTPClient.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index b23e8061291..1a23e95d648 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -166,15 +166,14 @@ void PocoHTTPClient::makeRequestInternal( for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt) { Poco::URI target_uri(uri); - - /// Reverse proxy can replace host header with resolved ip address instead of host name. - /// This can lead to request signature difference on S3 side. - auto session = makeHTTPSession(target_uri, timeouts, false); - + HTTPSessionPtr session; auto request_configuration = per_request_configuration(request); if (!request_configuration.proxyHost.empty()) { + /// Reverse proxy can replace host header with resolved ip address instead of host name. + /// This can lead to request signature difference on S3 side. + session = makeHTTPSession(target_uri, timeouts, false); bool use_tunnel = request_configuration.proxyScheme == Aws::Http::Scheme::HTTP && target_uri.getScheme() == "https"; session->setProxy( @@ -184,6 +183,11 @@ void PocoHTTPClient::makeRequestInternal( use_tunnel ); } + else + { + session = makeHTTPSession(target_uri, timeouts, true); + } + Poco::Net::HTTPRequest poco_request(Poco::Net::HTTPRequest::HTTP_1_1); From ec18340351a6cfd1ba683db8cedba5fa9b8d1275 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Oct 2021 19:11:50 +0300 Subject: [PATCH 453/950] Remove streams from formats. --- programs/client/Client.cpp | 3 - programs/copier/Internals.h | 1 - programs/library-bridge/Handlers.cpp | 31 ++- .../library-bridge/SharedLibraryHandler.cpp | 18 +- .../library-bridge/SharedLibraryHandler.h | 6 +- programs/obfuscator/Obfuscator.cpp | 22 +- programs/odbc-bridge/MainHandler.cpp | 26 ++- programs/odbc-bridge/ODBCBlockInputStream.cpp | 17 +- programs/odbc-bridge/ODBCBlockInputStream.h | 14 +- .../odbc-bridge/ODBCBlockOutputStream.cpp | 27 ++- programs/odbc-bridge/ODBCBlockOutputStream.h | 12 +- src/Bridge/LibraryBridgeHelper.cpp | 12 +- src/Client/ClientBase.cpp | 41 ++-- src/Client/ClientBase.h | 2 +- src/Client/Suggest.cpp | 1 - src/Common/RemoteHostFilter.cpp | 1 - src/Core/ExternalTable.cpp | 3 +- src/DataStreams/NullBlockOutputStream.h | 22 -- src/DataStreams/ShellCommandSource.h | 3 +- src/DataStreams/formatBlock.cpp | 15 +- src/DataStreams/formatBlock.h | 8 +- .../ExecutableDictionarySource.cpp | 4 +- .../ExecutablePoolDictionarySource.cpp | 4 +- src/Dictionaries/FileDictionarySource.cpp | 3 +- src/Dictionaries/HTTPDictionarySource.cpp | 11 +- src/Dictionaries/XDBCDictionarySource.cpp | 3 +- src/Formats/FormatFactory.cpp | 98 +-------- src/Formats/FormatFactory.h | 47 +---- src/Formats/NativeFormat.cpp | 4 +- src/Formats/NullFormat.cpp | 9 +- src/Formats/NullFormat.h | 20 ++ src/Formats/registerFormats.cpp | 188 +++++++++--------- src/Interpreters/Context.cpp | 14 +- src/Interpreters/Context.h | 9 +- .../UserDefinedExecutableFunctionFactory.cpp | 4 +- .../Formats/Impl/ArrowBlockInputFormat.cpp | 8 +- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 8 +- .../Formats/Impl/AvroRowInputFormat.cpp | 8 +- .../Formats/Impl/AvroRowOutputFormat.cpp | 6 +- .../Formats/Impl/BinaryRowInputFormat.cpp | 6 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 6 +- .../Formats/Impl/CSVRowInputFormat.cpp | 4 +- .../Formats/Impl/CSVRowOutputFormat.cpp | 4 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 6 +- .../Impl/JSONAsStringRowInputFormat.cpp | 4 +- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 10 +- .../JSONCompactEachRowRowOutputFormat.cpp | 10 +- .../Impl/JSONCompactRowOutputFormat.cpp | 6 +- .../Impl/JSONEachRowRowInputFormat.cpp | 6 +- .../Impl/JSONEachRowRowOutputFormat.cpp | 6 +- ...JSONEachRowWithProgressRowOutputFormat.cpp | 6 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 6 +- .../Impl/LineAsStringRowInputFormat.cpp | 4 +- .../Formats/Impl/MarkdownRowOutputFormat.cpp | 4 +- .../Formats/Impl/MsgPackRowInputFormat.cpp | 6 +- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 6 +- .../Formats/Impl/MySQLOutputFormat.cpp | 4 +- src/Processors/Formats/Impl/NullFormat.cpp | 4 +- .../Impl/ODBCDriver2BlockOutputFormat.cpp | 4 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 6 +- .../Formats/Impl/ORCBlockOutputFormat.cpp | 6 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 6 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 6 +- .../Formats/Impl/PostgreSQLOutputFormat.cpp | 4 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 6 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 6 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 6 +- .../Formats/Impl/ProtobufRowInputFormat.cpp | 6 +- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 6 +- .../Formats/Impl/RawBLOBRowInputFormat.cpp | 4 +- .../Formats/Impl/RawBLOBRowOutputFormat.cpp | 4 +- .../Formats/Impl/RegexpRowInputFormat.cpp | 4 +- .../Formats/Impl/TSKVRowInputFormat.cpp | 4 +- .../Formats/Impl/TSKVRowOutputFormat.cpp | 4 +- .../Impl/TabSeparatedRowInputFormat.cpp | 10 +- .../Impl/TabSeparatedRowOutputFormat.cpp | 10 +- .../Impl/TemplateBlockOutputFormat.cpp | 6 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 6 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 4 +- .../Formats/Impl/ValuesRowOutputFormat.cpp | 4 +- .../Formats/Impl/VerticalRowOutputFormat.cpp | 4 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 4 +- .../getSourceFromASTInsertQuery.cpp | 3 +- src/Server/GRPCServer.cpp | 38 ++-- src/Storages/HDFS/StorageHDFS.cpp | 11 +- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 9 +- src/Storages/Kafka/KafkaBlockOutputStream.h | 5 +- src/Storages/Kafka/KafkaSource.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/RabbitMQ/RabbitMQSink.cpp | 7 +- src/Storages/RabbitMQ/RabbitMQSink.h | 5 +- src/Storages/RabbitMQ/RabbitMQSource.cpp | 5 +- src/Storages/StorageExecutable.cpp | 3 +- src/Storages/StorageExternalDistributed.cpp | 1 - src/Storages/StorageFile.cpp | 13 +- src/Storages/StorageMySQL.cpp | 3 +- src/Storages/StorageNull.h | 1 - src/Storages/StoragePostgreSQL.cpp | 1 - src/Storages/StorageS3.cpp | 15 +- src/Storages/StorageS3Cluster.cpp | 1 - src/Storages/StorageSQLite.cpp | 3 +- src/Storages/StorageURL.cpp | 7 +- src/Storages/StorageURL.h | 5 +- src/Storages/System/StorageSystemFormats.cpp | 4 +- src/Storages/tests/gtest_storage_log.cpp | 14 +- src/TableFunctions/TableFunctionFile.cpp | 1 - src/TableFunctions/TableFunctionURL.cpp | 1 - 107 files changed, 519 insertions(+), 616 deletions(-) delete mode 100644 src/DataStreams/NullBlockOutputStream.h create mode 100644 src/Formats/NullFormat.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 04193036872..e0a97be4496 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -44,8 +44,6 @@ #include #include -#include - #include #include #include @@ -61,7 +59,6 @@ #include #include #include -#include #include "TestTags.h" #ifndef __clang__ diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 45531e750b9..9e648060592 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -52,7 +52,6 @@ #include #include #include -#include #include #include #include diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp index 2b6d0057bb2..2f6dca5ee65 100644 --- a/programs/library-bridge/Handlers.cpp +++ b/programs/library-bridge/Handlers.cpp @@ -11,6 +11,10 @@ #include #include #include +#include +#include +#include +#include #include #include @@ -63,6 +67,17 @@ namespace } +static void writeData(Block data, OutputFormatPtr format) +{ + auto source = std::make_shared(std::move(data)); + QueryPipeline pipeline(std::move(source)); + pipeline.complete(std::move(format)); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); +} + + void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) { LOG_TRACE(log, "Request URI: {}", request.getURI()); @@ -173,7 +188,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe } ReadBufferFromString read_block_buf(params.get("null_values")); - auto format = FormatFactory::instance().getInput(FORMAT, read_block_buf, *sample_block, getContext(), DEFAULT_BLOCK_SIZE); + auto format = getContext()->getInputFormat(FORMAT, read_block_buf, *sample_block, DEFAULT_BLOCK_SIZE); auto reader = std::make_shared(format); auto sample_block_with_nulls = reader->read(); @@ -221,8 +236,8 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe auto input = library_handler->loadAll(); LOG_DEBUG(log, "Started sending result data for dictionary id: {}", dictionary_id); - BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext()); - copyData(*input, *output); + auto output = FormatFactory::instance().getOutputFormat(FORMAT, out, sample_block, getContext()); + writeData(std::move(input), std::move(output)); } else if (method == "loadIds") { @@ -239,8 +254,8 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe auto input = library_handler->loadIds(ids); LOG_DEBUG(log, "Started sending result data for dictionary id: {}", dictionary_id); - BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext()); - copyData(*input, *output); + auto output = FormatFactory::instance().getOutputFormat(FORMAT, out, sample_block, getContext()); + writeData(std::move(input), std::move(output)); } else if (method == "loadKeys") { @@ -265,7 +280,7 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe } auto & read_buf = request.getStream(); - auto format = FormatFactory::instance().getInput(FORMAT, read_buf, *requested_sample_block, getContext(), DEFAULT_BLOCK_SIZE); + auto format = getContext()->getInputFormat(FORMAT, read_buf, *requested_sample_block, DEFAULT_BLOCK_SIZE); auto reader = std::make_shared(format); auto block = reader->read(); @@ -278,8 +293,8 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe auto input = library_handler->loadKeys(block.getColumns()); LOG_DEBUG(log, "Started sending result data for dictionary id: {}", dictionary_id); - BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream(FORMAT, out, sample_block, getContext()); - copyData(*input, *output); + auto output = FormatFactory::instance().getOutputFormat(FORMAT, out, sample_block, getContext()); + writeData(std::move(input), std::move(output)); } } catch (...) diff --git a/programs/library-bridge/SharedLibraryHandler.cpp b/programs/library-bridge/SharedLibraryHandler.cpp index c553f49e161..cfdba2c9eb8 100644 --- a/programs/library-bridge/SharedLibraryHandler.cpp +++ b/programs/library-bridge/SharedLibraryHandler.cpp @@ -92,7 +92,7 @@ bool SharedLibraryHandler::supportsSelectiveLoad() } -BlockInputStreamPtr SharedLibraryHandler::loadAll() +Block SharedLibraryHandler::loadAll() { auto columns_holder = std::make_unique(attributes_names.size()); ClickHouseLibrary::CStrings columns{static_cast(columns_holder.get()), attributes_names.size()}; @@ -107,13 +107,11 @@ BlockInputStreamPtr SharedLibraryHandler::loadAll() SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); ClickHouseLibrary::RawClickHouseLibraryTable data = load_all_func(data_ptr, &settings_holder->strings, &columns); - auto block = dataToBlock(data); - - return std::make_shared(block); + return dataToBlock(data); } -BlockInputStreamPtr SharedLibraryHandler::loadIds(const std::vector & ids) +Block SharedLibraryHandler::loadIds(const std::vector & ids) { const ClickHouseLibrary::VectorUInt64 ids_data{bit_cast(ids.data()), ids.size()}; @@ -128,13 +126,11 @@ BlockInputStreamPtr SharedLibraryHandler::loadIds(const std::vector & SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); ClickHouseLibrary::RawClickHouseLibraryTable data = load_ids_func(data_ptr, &settings_holder->strings, &columns_pass, &ids_data); - auto block = dataToBlock(data); - - return std::make_shared(block); + return dataToBlock(data); } -BlockInputStreamPtr SharedLibraryHandler::loadKeys(const Columns & key_columns) +Block SharedLibraryHandler::loadKeys(const Columns & key_columns) { auto holder = std::make_unique(key_columns.size()); std::vector> column_data_holders; @@ -171,9 +167,7 @@ BlockInputStreamPtr SharedLibraryHandler::loadKeys(const Columns & key_columns) SCOPE_EXIT(data_delete_func(lib_data, data_ptr)); ClickHouseLibrary::RawClickHouseLibraryTable data = load_keys_func(data_ptr, &settings_holder->strings, &request_cols); - auto block = dataToBlock(data); - - return std::make_shared(block); + return dataToBlock(data); } diff --git a/programs/library-bridge/SharedLibraryHandler.h b/programs/library-bridge/SharedLibraryHandler.h index 1dc923467d9..f9d2fe43cb2 100644 --- a/programs/library-bridge/SharedLibraryHandler.h +++ b/programs/library-bridge/SharedLibraryHandler.h @@ -27,11 +27,11 @@ public: ~SharedLibraryHandler(); - BlockInputStreamPtr loadAll(); + Block loadAll(); - BlockInputStreamPtr loadIds(const std::vector & ids); + Block loadIds(const std::vector & ids); - BlockInputStreamPtr loadKeys(const Columns & key_columns); + Block loadKeys(const Columns & key_columns); bool isModified(); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 9d5e03d871e..f18281a9fdf 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -24,10 +24,10 @@ #include #include #include -#include #include #include #include +#include #include #include #include @@ -1160,7 +1160,7 @@ try if (!silent) std::cerr << "Training models\n"; - Pipe pipe(FormatFactory::instance().getInput(input_format, file_in, header, context, max_block_size)); + Pipe pipe(context->getInputFormat(input_format, file_in, header, max_block_size)); QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -1189,7 +1189,7 @@ try file_in.seek(0, SEEK_SET); - Pipe pipe(FormatFactory::instance().getInput(input_format, file_in, header, context, max_block_size)); + Pipe pipe(context->getInputFormat(input_format, file_in, header, max_block_size)); if (processed_rows + source_rows > limit) { @@ -1199,23 +1199,25 @@ try }); } - QueryPipeline pipeline(std::move(pipe)); + QueryPipeline in_pipeline(std::move(pipe)); - BlockOutputStreamPtr output = context->getOutputStreamParallelIfPossible(output_format, file_out, header); + auto output = context->getOutputFormatParallelIfPossible(output_format, file_out, header); + QueryPipeline out_pipeline(std::move(output)); - PullingPipelineExecutor executor(pipeline); + PullingPipelineExecutor in_executor(in_pipeline); + PushingPipelineExecutor out_executor(out_pipeline); - output->writePrefix(); Block block; - while (executor.pull(block)) + out_executor.start(); + while (in_executor.pull(block)) { Columns columns = obfuscator.generate(block.getColumns()); - output->write(header.cloneWithColumns(columns)); + out_executor.push(header.cloneWithColumns(columns)); processed_rows += block.rows(); if (!silent) std::cerr << "Processed " << processed_rows << " rows\n"; } - output->writeSuffix(); + out_executor.finish(); obfuscator.updateSeed(); } diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index da53dcd75fb..51abe207095 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -16,6 +16,8 @@ #include #include #include +#include +#include #include #include @@ -133,10 +135,15 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse quoting_style = getQuotingStyle(connection_handler); #endif auto & read_buf = request.getStream(); - auto input_format = FormatFactory::instance().getInput(format, read_buf, *sample_block, getContext(), max_block_size); - auto input_stream = std::make_shared(input_format); - ODBCBlockOutputStream output_stream(std::move(connection_handler), db_name, table_name, *sample_block, getContext(), quoting_style); - copyData(*input_stream, output_stream); + auto input_format = getContext()->getInputFormat(format, read_buf, *sample_block, max_block_size); + auto sink = std::make_shared(std::move(connection_handler), db_name, table_name, *sample_block, getContext(), quoting_style); + + QueryPipeline pipeline(std::move(input_format)); + pipeline.complete(std::move(sink)); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + writeStringBinary("Ok.", out); } else @@ -144,9 +151,14 @@ void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse std::string query = params.get("query"); LOG_TRACE(log, "Query: {}", query); - BlockOutputStreamPtr writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, out, *sample_block, getContext()); - ODBCBlockInputStream inp(std::move(connection_handler), query, *sample_block, max_block_size); - copyData(inp, *writer); + auto writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, out, *sample_block, getContext()); + auto source = std::make_shared(std::move(connection_handler), query, *sample_block, max_block_size); + + QueryPipeline pipeline(std::move(source)); + pipeline.complete(std::move(writer)); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); } } catch (...) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 53633b3d293..110df30060d 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -19,9 +19,10 @@ namespace ErrorCodes } -ODBCBlockInputStream::ODBCBlockInputStream( +ODBCSource::ODBCSource( nanodbc::ConnectionHolderPtr connection_holder, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_) - : log(&Poco::Logger::get("ODBCBlockInputStream")) + : ISource(sample_block) + , log(&Poco::Logger::get("ODBCBlockInputStream")) , max_block_size{max_block_size_} , query(query_str) { @@ -31,10 +32,10 @@ ODBCBlockInputStream::ODBCBlockInputStream( } -Block ODBCBlockInputStream::readImpl() +Chunk ODBCSource::generate() { - if (finished) - return Block(); + if (is_finished) + return {}; MutableColumns columns(description.sample_block.cloneEmptyColumns()); size_t num_rows = 0; @@ -43,7 +44,7 @@ Block ODBCBlockInputStream::readImpl() { if (!result.next()) { - finished = true; + is_finished = true; break; } @@ -75,11 +76,11 @@ Block ODBCBlockInputStream::readImpl() break; } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -void ODBCBlockInputStream::insertValue( +void ODBCSource::insertValue( IColumn & column, const DataTypePtr data_type, const ValueType type, nanodbc::result & row, size_t idx) { switch (type) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.h b/programs/odbc-bridge/ODBCBlockInputStream.h index 26aa766dbcc..84d270e4713 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.h +++ b/programs/odbc-bridge/ODBCBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include "ODBCConnectionFactory.h" @@ -10,22 +10,20 @@ namespace DB { /// Allows processing results of a query to ODBC source as a sequence of Blocks, simplifies chaining -class ODBCBlockInputStream final : public IBlockInputStream +class ODBCSource final : public ISource { public: - ODBCBlockInputStream(nanodbc::ConnectionHolderPtr connection, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_); + ODBCSource(nanodbc::ConnectionHolderPtr connection, const std::string & query_str, const Block & sample_block, UInt64 max_block_size_); String getName() const override { return "ODBC"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: using QueryResult = std::shared_ptr; using ValueType = ExternalResultDescription::ValueType; - Block readImpl() override; + Chunk generate() override; - static void insertValue(IColumn & column, const DataTypePtr data_type, const ValueType type, nanodbc::result & row, size_t idx); + static void insertValue(IColumn & column, DataTypePtr data_type, ValueType type, nanodbc::result & row, size_t idx); static void insertDefaultValue(IColumn & column, const IColumn & sample_column) { @@ -38,7 +36,7 @@ private: nanodbc::result result; String query; - bool finished = false; + bool is_finished = false; }; } diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 32ceea3aef8..0e6770c1295 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -8,7 +8,7 @@ #include "getIdentifierQuote.h" #include #include -#include +#include #include @@ -16,13 +16,15 @@ namespace DB { -ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::ConnectionHolderPtr connection_holder_, - const std::string & remote_database_name_, - const std::string & remote_table_name_, - const Block & sample_block_, - ContextPtr local_context_, - IdentifierQuotingStyle quoting_) - : log(&Poco::Logger::get("ODBCBlockOutputStream")) +ODBCSink::ODBCSink( + nanodbc::ConnectionHolderPtr connection_holder_, + const std::string & remote_database_name_, + const std::string & remote_table_name_, + const Block & sample_block_, + ContextPtr local_context_, + IdentifierQuotingStyle quoting_) + : ISink(sample_block_) + , log(&Poco::Logger::get("ODBCBlockOutputStream")) , connection_holder(std::move(connection_holder_)) , db_name(remote_database_name_) , table_name(remote_table_name_) @@ -33,15 +35,12 @@ ODBCBlockOutputStream::ODBCBlockOutputStream(nanodbc::ConnectionHolderPtr connec description.init(sample_block); } -Block ODBCBlockOutputStream::getHeader() const -{ - return sample_block; -} -void ODBCBlockOutputStream::write(const Block & block) +void ODBCSink::consume(Chunk chunk) { + auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); WriteBufferFromOwnString values_buf; - auto writer = FormatFactory::instance().getOutputStream("Values", values_buf, sample_block, local_context); + auto writer = local_context->getOutputFormat("Values", values_buf, sample_block); writer->write(block); std::string query = getInsertQuery(db_name, table_name, block.getColumnsWithTypeAndName(), quoting) + values_buf.str(); diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.h b/programs/odbc-bridge/ODBCBlockOutputStream.h index 16a1602d3cd..2c57b8ee84f 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.h +++ b/programs/odbc-bridge/ODBCBlockOutputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -11,12 +11,12 @@ namespace DB { -class ODBCBlockOutputStream : public IBlockOutputStream +class ODBCSink final : public ISink { using ValueType = ExternalResultDescription::ValueType; public: - ODBCBlockOutputStream( + ODBCSink( nanodbc::ConnectionHolderPtr connection_, const std::string & remote_database_name_, const std::string & remote_table_name_, @@ -24,8 +24,10 @@ public: ContextPtr local_context_, IdentifierQuotingStyle quoting); - Block getHeader() const override; - void write(const Block & block) override; + String getName() const override { return "ODBCSink"; } + +protected: + void consume(Chunk chunk) override; private: Poco::Logger * log; diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 7f3e3f8009c..af84daf3da0 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -134,8 +133,8 @@ ReadWriteBufferFromHTTP::OutStreamCallback LibraryBridgeHelper::getInitLibraryCa { /// Sample block must contain null values WriteBufferFromOwnString out; - auto output_stream = getContext()->getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out, sample_block); - formatBlock(output_stream, sample_block); + auto output_format = getContext()->getOutputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, out, sample_block); + formatBlock(output_format, sample_block); auto block_string = out.str(); return [block_string, this](std::ostream & os) @@ -226,8 +225,8 @@ Pipe LibraryBridgeHelper::loadKeys(const Block & requested_block) ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [requested_block, this](std::ostream & os) { WriteBufferFromOStream out_buffer(os); - auto output_stream = getContext()->getOutputStream(LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block); - formatBlock(output_stream, requested_block); + auto output_format = getContext()->getOutputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block); + formatBlock(output_format, requested_block); }; return loadBase(uri, out_stream_callback); } @@ -259,8 +258,7 @@ Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTT DBMS_DEFAULT_BUFFER_SIZE, ReadWriteBufferFromHTTP::HTTPHeaderEntries{}); - auto input_stream = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); - auto source = FormatFactory::instance().getInput(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE); + auto source = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); source->addBuffer(std::move(read_buf_ptr)); return Pipe(std::move(source)); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 56d9993d14b..99a80901cdd 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -38,16 +38,15 @@ #include #include -#include +#include #include +#include #include #include #include #include #include #include - -#include #include namespace fs = std::filesystem; @@ -233,7 +232,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) initBlockOutputStream(block, parsed_query); /// The header block containing zero rows was used to initialize - /// block_out_stream, do not output it. + /// output_format, do not output it. /// Also do not output too much data if we're fuzzing. if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) return; @@ -241,11 +240,11 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (need_render_progress && (stdout_is_a_tty || is_interactive)) progress_indication.clearProgressOutput(); - block_out_stream->write(block); + output_format->write(block); written_first_block = true; /// Received data block is immediately displayed to the user. - block_out_stream->flush(); + output_format->flush(); /// Restore progress bar after data block. if (need_render_progress && (stdout_is_a_tty || is_interactive)) @@ -265,14 +264,14 @@ void ClientBase::onLogData(Block & block) void ClientBase::onTotals(Block & block, ASTPtr parsed_query) { initBlockOutputStream(block, parsed_query); - block_out_stream->setTotals(block); + output_format->setTotals(block); } void ClientBase::onExtremes(Block & block, ASTPtr parsed_query) { initBlockOutputStream(block, parsed_query); - block_out_stream->setExtremes(block); + output_format->setExtremes(block); } @@ -286,19 +285,19 @@ void ClientBase::onReceiveExceptionFromServer(std::unique_ptr && e) void ClientBase::onProfileInfo(const BlockStreamProfileInfo & profile_info) { - if (profile_info.hasAppliedLimit() && block_out_stream) - block_out_stream->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); + if (profile_info.hasAppliedLimit() && output_format) + output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); } void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) { - if (!block_out_stream) + if (!output_format) { /// Ignore all results when fuzzing as they can be huge. if (query_fuzzer_runs) { - block_out_stream = std::make_shared(block); + output_format = std::make_shared(block); return; } @@ -360,11 +359,11 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) /// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly. if (!need_render_progress) - block_out_stream = global_context->getOutputStreamParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + output_format = global_context->getOutputFormatParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); else - block_out_stream = global_context->getOutputStream(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + output_format = global_context->getOutputFormat(current_format, out_file_buf ? *out_file_buf : *out_buf, block); - block_out_stream->writePrefix(); + output_format->doWritePrefix(); } } @@ -626,8 +625,8 @@ void ClientBase::onProgress(const Progress & value) return; } - if (block_out_stream) - block_out_stream->onProgress(value); + if (output_format) + output_format->onProgress(value); if (need_render_progress) progress_indication.writeProgress(); @@ -638,8 +637,8 @@ void ClientBase::onEndOfStream() { progress_indication.clearProgressOutput(); - if (block_out_stream) - block_out_stream->writeSuffix(); + if (output_format) + output_format->doWriteSuffix(); resetOutput(); @@ -654,7 +653,7 @@ void ClientBase::onEndOfStream() /// Flush all buffers. void ClientBase::resetOutput() { - block_out_stream.reset(); + output_format.reset(); logs_out_stream.reset(); if (pager_cmd) @@ -848,7 +847,7 @@ void ClientBase::sendDataFrom(ReadBuffer & buf, Block & sample, const ColumnsDes current_format = insert->format; } - auto source = FormatFactory::instance().getInput(current_format, buf, sample, global_context, insert_format_max_block_size); + auto source = global_context->getInputFormat(current_format, buf, sample, insert_format_max_block_size); Pipe pipe(source); if (columns_description.hasDefaults()) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index b122803e1db..92f9d48e070 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -177,7 +177,7 @@ protected: /// The user can specify to redirect query output to a file. std::unique_ptr out_file_buf; - BlockOutputStreamPtr block_out_stream; + std::shared_ptr output_format; /// The user could specify special file for server logs (stderr by default) std::unique_ptr out_logs_buf; diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index f500332b616..4796f7f9cf1 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Common/RemoteHostFilter.cpp b/src/Common/RemoteHostFilter.cpp index 6ea366314e1..73c84364f3c 100644 --- a/src/Common/RemoteHostFilter.cpp +++ b/src/Common/RemoteHostFilter.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 6ae89de1276..55ad748868e 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -19,7 +19,6 @@ #include #include -#include #include @@ -36,7 +35,7 @@ ExternalTableDataPtr BaseExternalTable::getData(ContextPtr context) { initReadBuffer(); initSampleBlock(); - auto input = FormatFactory::instance().getInput(format, *read_buffer, sample_block, context, DEFAULT_BLOCK_SIZE); + auto input = context->getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE); auto data = std::make_unique(); data->pipe = std::make_unique(std::move(input)); diff --git a/src/DataStreams/NullBlockOutputStream.h b/src/DataStreams/NullBlockOutputStream.h deleted file mode 100644 index 8b3e61d35a7..00000000000 --- a/src/DataStreams/NullBlockOutputStream.h +++ /dev/null @@ -1,22 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/** Does nothing. Used for debugging and benchmarks. - */ -class NullBlockOutputStream : public IBlockOutputStream -{ -public: - NullBlockOutputStream(const Block & header_) : header(header_) {} - Block getHeader() const override { return header; } - void write(const Block &) override {} - -private: - Block header; -}; - -} diff --git a/src/DataStreams/ShellCommandSource.h b/src/DataStreams/ShellCommandSource.h index 6df233639a7..18dbd2e03aa 100644 --- a/src/DataStreams/ShellCommandSource.h +++ b/src/DataStreams/ShellCommandSource.h @@ -9,7 +9,6 @@ #include #include -#include #include #include #include @@ -97,7 +96,7 @@ public: max_block_size = configuration.number_of_rows_to_read; } - pipeline = QueryPipeline(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size))); + pipeline = QueryPipeline(Pipe(context->getInputFormat(format, command->out, sample_block, max_block_size))); executor = std::make_unique(pipeline); } diff --git a/src/DataStreams/formatBlock.cpp b/src/DataStreams/formatBlock.cpp index ec98e4e98da..dab321be2e1 100644 --- a/src/DataStreams/formatBlock.cpp +++ b/src/DataStreams/formatBlock.cpp @@ -1,15 +1,20 @@ #include -#include #include +#include +#include +#include +#include namespace DB { -void formatBlock(BlockOutputStreamPtr & out, const Block & block) +void formatBlock(OutputFormatPtr out, const Block & block) { - out->writePrefix(); - out->write(block); - out->writeSuffix(); + auto source = std::make_shared(block); + QueryPipeline pipeline(source); + pipeline.complete(out); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); out->flush(); } diff --git a/src/DataStreams/formatBlock.h b/src/DataStreams/formatBlock.h index 57cc5fd77bf..fd206a23a45 100644 --- a/src/DataStreams/formatBlock.h +++ b/src/DataStreams/formatBlock.h @@ -1,12 +1,14 @@ #pragma once - -#include +#include namespace DB { class Block; -void formatBlock(BlockOutputStreamPtr & out, const Block & block); +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + +void formatBlock(OutputFormatPtr out, const Block & block); } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 58aab562cde..91aeda924a1 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -127,8 +127,8 @@ Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) writeChar('\n', out); } - auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); - formatBlock(output_stream, block); + auto output_format = context->getOutputFormat(configuration.format, out, block.cloneEmpty()); + formatBlock(output_format, block); out.close(); }}; std::vector tasks = {std::move(task)}; diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index c7859aab9bd..79e9b627836 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -119,8 +119,8 @@ Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) writeChar('\n', out); } - auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); - formatBlock(output_stream, block); + auto output_format = context->getOutputFormat(configuration.format, out, block.cloneEmpty()); + formatBlock(output_format, block); }; std::vector tasks = {std::move(task)}; diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 1138d28c4c9..8c1f099f344 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" @@ -51,7 +50,7 @@ Pipe FileDictionarySource::loadAll() { LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString()); auto in_ptr = std::make_unique(filepath); - auto source = FormatFactory::instance().getInput(format, *in_ptr, sample_block, context, max_block_size); + auto source = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); source->addBuffer(std::move(in_ptr)); last_modification = getLastModification(); diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index d6a0c25e9f1..6cb4d52744a 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -69,7 +68,7 @@ Pipe HTTPDictionarySource::createWrappedBuffer(std::unique_ptrgetCompressionMethod(); auto in_ptr_wrapped = wrapReadBufferWithCompressionMethod(std::move(http_buffer_ptr), chooseCompressionMethod(uri.getPath(), http_request_compression_method_str)); - auto source = FormatFactory::instance().getInput(configuration.format, *in_ptr_wrapped, sample_block, context, max_block_size); + auto source = context->getInputFormat(configuration.format, *in_ptr_wrapped, sample_block, max_block_size); source->addBuffer(std::move(in_ptr_wrapped)); return Pipe(std::move(source)); } @@ -135,8 +134,8 @@ Pipe HTTPDictionarySource::loadIds(const std::vector & ids) ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); - auto output_stream = context->getOutputStreamParallelIfPossible(configuration.format, out_buffer, sample_block); - formatBlock(output_stream, block); + auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, sample_block); + formatBlock(output_format, block); }; Poco::URI uri(configuration.url); @@ -162,8 +161,8 @@ Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vect ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); - auto output_stream = context->getOutputStreamParallelIfPossible(configuration.format, out_buffer, sample_block); - formatBlock(output_stream, block); + auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, sample_block); + formatBlock(output_format, block); }; Poco::URI uri(configuration.url); diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 12831b7a970..80081e67b42 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -216,7 +215,7 @@ Pipe XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & re }; auto read_buf = std::make_unique(url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts); - auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, getContext(), max_block_size); + auto format = getContext()->getInputFormat(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, required_sample_block, max_block_size); format->addBuffer(std::move(read_buf)); return Pipe(std::move(format)); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9901081d7dd..d3ff5cbf8a7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -140,7 +140,7 @@ InputFormatPtr FormatFactory::getInput( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - if (!getCreators(name).input_processor_creator) + if (!getCreators(name).input_creator) { throw Exception("Format " + name + " is not suitable for input (with processors)", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); } @@ -168,7 +168,7 @@ InputFormatPtr FormatFactory::getInput( if (parallel_parsing) { - const auto & input_getter = getCreators(name).input_processor_creator; + const auto & input_getter = getCreators(name).input_creator; RowInputFormatParams row_input_format_params; row_input_format_params.max_block_size = max_block_size; @@ -193,69 +193,6 @@ InputFormatPtr FormatFactory::getInput( return format; } -BlockOutputStreamPtr FormatFactory::getOutputStreamParallelIfPossible( - const String & name, - WriteBuffer & buf, - const Block & sample, - ContextPtr context, - WriteCallback callback, - const std::optional & _format_settings) const -{ - const auto & output_getter = getCreators(name).output_processor_creator; - - const Settings & settings = context->getSettingsRef(); - bool parallel_formatting = settings.output_format_parallel_formatting; - auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - - if (output_getter && parallel_formatting && getCreators(name).supports_parallel_formatting - && !settings.output_format_json_array_of_rows) - { - auto formatter_creator = [output_getter, sample, callback, format_settings] - (WriteBuffer & output) -> OutputFormatPtr - { return output_getter(output, sample, {std::move(callback)}, format_settings);}; - - ParallelFormattingOutputFormat::Params params{buf, sample, formatter_creator, settings.max_threads}; - auto format = std::make_shared(params); - - /// Enable auto-flush for streaming mode. Currently it is needed by INSERT WATCH query. - if (format_settings.enable_streaming) - format->setAutoFlush(); - - return std::make_shared(std::make_shared(format), sample); - } - - return getOutputStream(name, buf, sample, context, callback, _format_settings); -} - - -BlockOutputStreamPtr FormatFactory::getOutputStream( - const String & name, - WriteBuffer & buf, - const Block & sample, - ContextPtr context, - WriteCallback callback, - const std::optional & _format_settings) const -{ - auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - - if (!getCreators(name).output_processor_creator) - { - const auto & output_getter = getCreators(name).output_creator; - if (!output_getter) - throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT); - - /** Materialization is needed, because formats can use the functions `IDataType`, - * which only work with full columns. - */ - return std::make_shared( - output_getter(buf, sample, std::move(callback), format_settings), - sample); - } - - auto format = getOutputFormat(name, buf, sample, context, std::move(callback), _format_settings); - return std::make_shared(std::make_shared(format), sample); -} - InputFormatPtr FormatFactory::getInputFormat( const String & name, @@ -265,7 +202,7 @@ InputFormatPtr FormatFactory::getInputFormat( UInt64 max_block_size, const std::optional & _format_settings) const { - const auto & input_getter = getCreators(name).input_processor_creator; + const auto & input_getter = getCreators(name).input_creator; if (!input_getter) throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); @@ -299,7 +236,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible( WriteCallback callback, const std::optional & _format_settings) const { - const auto & output_getter = getCreators(name).output_processor_creator; + const auto & output_getter = getCreators(name).output_creator; if (!output_getter) throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT, "Format {} is not suitable for output (with processors)", name); @@ -334,7 +271,7 @@ OutputFormatPtr FormatFactory::getOutputFormat( WriteCallback callback, const std::optional & _format_settings) const { - const auto & output_getter = getCreators(name).output_processor_creator; + const auto & output_getter = getCreators(name).output_creator; if (!output_getter) throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT, "Format {} is not suitable for output (with processors)", name); @@ -362,7 +299,6 @@ OutputFormatPtr FormatFactory::getOutputFormat( return format; } - void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator) { auto & target = dict[name].input_creator; @@ -371,22 +307,6 @@ void FormatFactory::registerInputFormat(const String & name, InputCreator input_ target = std::move(input_creator); } -void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator) -{ - auto & target = dict[name].output_creator; - if (target) - throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); - target = std::move(output_creator); -} - -void FormatFactory::registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator) -{ - auto & target = dict[name].input_processor_creator; - if (target) - throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); - target = std::move(input_creator); -} - void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker) { auto & target = dict[name].non_trivial_prefix_and_suffix_checker; @@ -395,9 +315,9 @@ void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name target = std::move(non_trivial_prefix_and_suffix_checker); } -void FormatFactory::registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator) +void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator) { - auto & target = dict[name].output_processor_creator; + auto & target = dict[name].output_creator; if (target) throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(output_creator); @@ -439,13 +359,13 @@ bool FormatFactory::checkIfFormatIsColumnOriented(const String & name) bool FormatFactory::isInputFormat(const String & name) const { auto it = dict.find(name); - return it != dict.end() && (it->second.input_creator || it->second.input_processor_creator); + return it != dict.end() && it->second.input_creator; } bool FormatFactory::isOutputFormat(const String & name) const { auto it = dict.find(name); - return it != dict.end() && (it->second.output_creator || it->second.output_processor_creator); + return it != dict.end() && it->second.output_creator; } FormatFactory & FormatFactory::instance() diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 0fcb8d042c7..d816ef1d016 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -66,28 +66,16 @@ public: size_t row)>; private: - using InputCreator = std::function; - using OutputCreator = std::function; - - using InputProcessorCreatorFunc = InputFormatPtr( + using InputCreatorFunc = InputFormatPtr( ReadBuffer & buf, const Block & header, const RowInputFormatParams & params, const FormatSettings & settings); - using InputProcessorCreator = std::function; + using InputCreator = std::function; - using OutputProcessorCreator = std::function & format_settings = std::nullopt) const; - /// Checks all preconditions. Returns ordinary stream if parallel formatting cannot be done. - /// Currently used only in Client. Don't use it something else! Better look at getOutputFormatParallelIfPossible. - BlockOutputStreamPtr getOutputStreamParallelIfPossible( - const String & name, - WriteBuffer & buf, - const Block & sample, - ContextPtr context, - WriteCallback callback = {}, - const std::optional & format_settings = std::nullopt) const; - - /// Currently used only in Client. Don't use it something else! Better look at getOutputFormat. - BlockOutputStreamPtr getOutputStream( - const String & name, - WriteBuffer & buf, - const Block & sample, - ContextPtr context, - WriteCallback callback = {}, - const std::optional & format_settings = std::nullopt) const; - InputFormatPtr getInputFormat( const String & name, ReadBuffer & buf, @@ -167,15 +134,13 @@ public: WriteCallback callback = {}, const std::optional & format_settings = std::nullopt) const; - /// Register format by its name. - void registerInputFormat(const String & name, InputCreator input_creator); - void registerOutputFormat(const String & name, OutputCreator output_creator); void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine); void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker); - void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator); - void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator); + /// Register format by its name. + void registerInputFormat(const String & name, InputCreator input_creator); + void registerOutputFormat(const String & name, OutputCreator output_creator); void markOutputFormatSupportsParallelFormatting(const String & name); void markFormatAsColumnOriented(const String & name); diff --git a/src/Formats/NativeFormat.cpp b/src/Formats/NativeFormat.cpp index b94cb279412..5daa38967b9 100644 --- a/src/Formats/NativeFormat.cpp +++ b/src/Formats/NativeFormat.cpp @@ -84,7 +84,7 @@ private: void registerInputFormatNative(FormatFactory & factory) { - factory.registerInputFormatProcessor("Native", []( + factory.registerInputFormat("Native", []( ReadBuffer & buf, const Block & sample, const RowInputFormatParams &, @@ -96,7 +96,7 @@ void registerInputFormatNative(FormatFactory & factory) void registerOutputFormatNative(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Native", []( + factory.registerOutputFormat("Native", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, diff --git a/src/Formats/NullFormat.cpp b/src/Formats/NullFormat.cpp index c07723211eb..8613293cb13 100644 --- a/src/Formats/NullFormat.cpp +++ b/src/Formats/NullFormat.cpp @@ -1,19 +1,22 @@ -#include +#include #include +#include namespace DB { +WriteBuffer NullOutputFormat::empty_buffer(nullptr, 0); + void registerOutputFormatNull(FormatFactory & factory) { factory.registerOutputFormat("Null", []( WriteBuffer &, const Block & sample, - FormatFactory::WriteCallback, + const RowOutputFormatParams &, const FormatSettings &) { - return std::make_shared(sample); + return std::make_shared(sample); }); } diff --git a/src/Formats/NullFormat.h b/src/Formats/NullFormat.h new file mode 100644 index 00000000000..9822c9f5d16 --- /dev/null +++ b/src/Formats/NullFormat.h @@ -0,0 +1,20 @@ +#include + +namespace DB +{ + +class NullOutputFormat final : public IOutputFormat +{ +public: + explicit NullOutputFormat(const Block & header) : IOutputFormat(header, empty_buffer) {} + + String getName() const override { return "Null"; } + +protected: + void consume(Chunk) override {} + +private: + static WriteBuffer empty_buffer; +}; + +} diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 2686022980f..a98876c15bd 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -21,61 +21,61 @@ void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); void registerInputFormatNative(FormatFactory & factory); void registerOutputFormatNative(FormatFactory & factory); -void registerInputFormatProcessorRowBinary(FormatFactory & factory); -void registerOutputFormatProcessorRowBinary(FormatFactory & factory); -void registerInputFormatProcessorTabSeparated(FormatFactory & factory); -void registerOutputFormatProcessorTabSeparated(FormatFactory & factory); -void registerInputFormatProcessorValues(FormatFactory & factory); -void registerOutputFormatProcessorValues(FormatFactory & factory); -void registerInputFormatProcessorCSV(FormatFactory & factory); -void registerOutputFormatProcessorCSV(FormatFactory & factory); -void registerInputFormatProcessorTSKV(FormatFactory & factory); -void registerOutputFormatProcessorTSKV(FormatFactory & factory); -void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); -void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); -void registerInputFormatProcessorProtobuf(FormatFactory & factory); -void registerOutputFormatProcessorProtobuf(FormatFactory & factory); -void registerInputFormatProcessorTemplate(FormatFactory & factory); -void registerOutputFormatProcessorTemplate(FormatFactory & factory); -void registerInputFormatProcessorMsgPack(FormatFactory & factory); -void registerOutputFormatProcessorMsgPack(FormatFactory & factory); -void registerInputFormatProcessorORC(FormatFactory & factory); -void registerOutputFormatProcessorORC(FormatFactory & factory); -void registerInputFormatProcessorParquet(FormatFactory & factory); -void registerOutputFormatProcessorParquet(FormatFactory & factory); -void registerInputFormatProcessorArrow(FormatFactory & factory); -void registerOutputFormatProcessorArrow(FormatFactory & factory); -void registerInputFormatProcessorAvro(FormatFactory & factory); -void registerOutputFormatProcessorAvro(FormatFactory & factory); -void registerInputFormatProcessorRawBLOB(FormatFactory & factory); -void registerOutputFormatProcessorRawBLOB(FormatFactory & factory); +void registerInputFormatRowBinary(FormatFactory & factory); +void registerOutputFormatRowBinary(FormatFactory & factory); +void registerInputFormatTabSeparated(FormatFactory & factory); +void registerOutputFormatTabSeparated(FormatFactory & factory); +void registerInputFormatValues(FormatFactory & factory); +void registerOutputFormatValues(FormatFactory & factory); +void registerInputFormatCSV(FormatFactory & factory); +void registerOutputFormatCSV(FormatFactory & factory); +void registerInputFormatTSKV(FormatFactory & factory); +void registerOutputFormatTSKV(FormatFactory & factory); +void registerInputFormatJSONEachRow(FormatFactory & factory); +void registerOutputFormatJSONEachRow(FormatFactory & factory); +void registerInputFormatJSONCompactEachRow(FormatFactory & factory); +void registerOutputFormatJSONCompactEachRow(FormatFactory & factory); +void registerInputFormatProtobuf(FormatFactory & factory); +void registerOutputFormatProtobuf(FormatFactory & factory); +void registerInputFormatTemplate(FormatFactory & factory); +void registerOutputFormatTemplate(FormatFactory & factory); +void registerInputFormatMsgPack(FormatFactory & factory); +void registerOutputFormatMsgPack(FormatFactory & factory); +void registerInputFormatORC(FormatFactory & factory); +void registerOutputFormatORC(FormatFactory & factory); +void registerInputFormatParquet(FormatFactory & factory); +void registerOutputFormatParquet(FormatFactory & factory); +void registerInputFormatArrow(FormatFactory & factory); +void registerOutputFormatArrow(FormatFactory & factory); +void registerInputFormatAvro(FormatFactory & factory); +void registerOutputFormatAvro(FormatFactory & factory); +void registerInputFormatRawBLOB(FormatFactory & factory); +void registerOutputFormatRawBLOB(FormatFactory & factory); /// Output only (presentational) formats. void registerOutputFormatNull(FormatFactory & factory); -void registerOutputFormatProcessorPretty(FormatFactory & factory); -void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory); -void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); -void registerOutputFormatProcessorVertical(FormatFactory & factory); -void registerOutputFormatProcessorJSON(FormatFactory & factory); -void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); -void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); -void registerOutputFormatProcessorXML(FormatFactory & factory); -void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); -void registerOutputFormatProcessorNull(FormatFactory & factory); -void registerOutputFormatProcessorMySQLWire(FormatFactory & factory); -void registerOutputFormatProcessorMarkdown(FormatFactory & factory); -void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory); +void registerOutputFormatPretty(FormatFactory & factory); +void registerOutputFormatPrettyCompact(FormatFactory & factory); +void registerOutputFormatPrettySpace(FormatFactory & factory); +void registerOutputFormatVertical(FormatFactory & factory); +void registerOutputFormatJSON(FormatFactory & factory); +void registerOutputFormatJSONCompact(FormatFactory & factory); +void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory); +void registerOutputFormatXML(FormatFactory & factory); +void registerOutputFormatODBCDriver2(FormatFactory & factory); +void registerOutputFormatNull(FormatFactory & factory); +void registerOutputFormatMySQLWire(FormatFactory & factory); +void registerOutputFormatMarkdown(FormatFactory & factory); +void registerOutputFormatPostgreSQLWire(FormatFactory & factory); /// Input only formats. -void registerInputFormatProcessorRegexp(FormatFactory & factory); -void registerInputFormatProcessorJSONAsString(FormatFactory & factory); -void registerInputFormatProcessorLineAsString(FormatFactory & factory); -void registerInputFormatProcessorCapnProto(FormatFactory & factory); +void registerInputFormatRegexp(FormatFactory & factory); +void registerInputFormatJSONAsString(FormatFactory & factory); +void registerInputFormatLineAsString(FormatFactory & factory); +void registerInputFormatCapnProto(FormatFactory & factory); /// Non trivial prefix and suffix checkers for disabling parallel parsing. void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory); @@ -94,62 +94,60 @@ void registerFormats() registerInputFormatNative(factory); registerOutputFormatNative(factory); - registerInputFormatProcessorRowBinary(factory); - registerOutputFormatProcessorRowBinary(factory); - registerInputFormatProcessorTabSeparated(factory); - registerOutputFormatProcessorTabSeparated(factory); - registerInputFormatProcessorValues(factory); - registerOutputFormatProcessorValues(factory); - registerInputFormatProcessorCSV(factory); - registerOutputFormatProcessorCSV(factory); - registerInputFormatProcessorTSKV(factory); - registerOutputFormatProcessorTSKV(factory); - registerInputFormatProcessorJSONEachRow(factory); - registerOutputFormatProcessorJSONEachRow(factory); - registerInputFormatProcessorJSONCompactEachRow(factory); - registerOutputFormatProcessorJSONCompactEachRow(factory); - registerInputFormatProcessorProtobuf(factory); - registerOutputFormatProcessorProtobuf(factory); - registerInputFormatProcessorTemplate(factory); - registerOutputFormatProcessorTemplate(factory); - registerInputFormatProcessorMsgPack(factory); - registerOutputFormatProcessorMsgPack(factory); - registerInputFormatProcessorRawBLOB(factory); - registerOutputFormatProcessorRawBLOB(factory); + registerInputFormatRowBinary(factory); + registerOutputFormatRowBinary(factory); + registerInputFormatTabSeparated(factory); + registerOutputFormatTabSeparated(factory); + registerInputFormatValues(factory); + registerOutputFormatValues(factory); + registerInputFormatCSV(factory); + registerOutputFormatCSV(factory); + registerInputFormatTSKV(factory); + registerOutputFormatTSKV(factory); + registerInputFormatJSONEachRow(factory); + registerOutputFormatJSONEachRow(factory); + registerInputFormatJSONCompactEachRow(factory); + registerOutputFormatJSONCompactEachRow(factory); + registerInputFormatProtobuf(factory); + registerOutputFormatProtobuf(factory); + registerInputFormatTemplate(factory); + registerOutputFormatTemplate(factory); + registerInputFormatMsgPack(factory); + registerOutputFormatMsgPack(factory); + registerInputFormatRawBLOB(factory); + registerOutputFormatRawBLOB(factory); - registerInputFormatProcessorORC(factory); - registerOutputFormatProcessorORC(factory); - registerInputFormatProcessorParquet(factory); - registerOutputFormatProcessorParquet(factory); + registerInputFormatORC(factory); + registerOutputFormatORC(factory); + registerInputFormatParquet(factory); + registerOutputFormatParquet(factory); #if !defined(ARCADIA_BUILD) - registerInputFormatProcessorAvro(factory); - registerOutputFormatProcessorAvro(factory); + registerInputFormatAvro(factory); + registerOutputFormatAvro(factory); #endif - registerInputFormatProcessorArrow(factory); - registerOutputFormatProcessorArrow(factory); + registerInputFormatArrow(factory); + registerOutputFormatArrow(factory); + registerOutputFormatPretty(factory); + registerOutputFormatPrettyCompact(factory); + registerOutputFormatPrettySpace(factory); + registerOutputFormatVertical(factory); + registerOutputFormatJSON(factory); + registerOutputFormatJSONCompact(factory); + registerOutputFormatJSONEachRowWithProgress(factory); + registerOutputFormatXML(factory); + registerOutputFormatODBCDriver2(factory); registerOutputFormatNull(factory); + registerOutputFormatMySQLWire(factory); + registerOutputFormatMarkdown(factory); + registerOutputFormatPostgreSQLWire(factory); - registerOutputFormatProcessorPretty(factory); - registerOutputFormatProcessorPrettyCompact(factory); - registerOutputFormatProcessorPrettySpace(factory); - registerOutputFormatProcessorVertical(factory); - registerOutputFormatProcessorJSON(factory); - registerOutputFormatProcessorJSONCompact(factory); - registerOutputFormatProcessorJSONEachRowWithProgress(factory); - registerOutputFormatProcessorXML(factory); - registerOutputFormatProcessorODBCDriver2(factory); - registerOutputFormatProcessorNull(factory); - registerOutputFormatProcessorMySQLWire(factory); - registerOutputFormatProcessorMarkdown(factory); - registerOutputFormatProcessorPostgreSQLWire(factory); - - registerInputFormatProcessorRegexp(factory); - registerInputFormatProcessorJSONAsString(factory); - registerInputFormatProcessorLineAsString(factory); + registerInputFormatRegexp(factory); + registerInputFormatJSONAsString(factory); + registerInputFormatLineAsString(factory); #if !defined(ARCADIA_BUILD) - registerInputFormatProcessorCapnProto(factory); + registerInputFormatCapnProto(factory); #endif registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1a95a642e18..6e729eb2370 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2428,20 +2428,14 @@ void Context::checkPartitionCanBeDropped(const String & database, const String & } -BlockInputStreamPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const +InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings) const { - return std::make_shared( - FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size)); + return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings); } -BlockOutputStreamPtr Context::getOutputStreamParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const +OutputFormatPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const { - return FormatFactory::instance().getOutputStreamParallelIfPossible(name, buf, sample, shared_from_this()); -} - -BlockOutputStreamPtr Context::getOutputStream(const String & name, WriteBuffer & buf, const Block & sample) const -{ - return FormatFactory::instance().getOutputStream(name, buf, sample, shared_from_this()); + return FormatFactory::instance().getOutputFormat(name, buf, sample, shared_from_this()); } OutputFormatPtr Context::getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f6cded2b1d1..1e19c18de43 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -116,7 +116,9 @@ using PartUUIDsPtr = std::shared_ptr; class KeeperDispatcher; class Session; +class IInputFormat; class IOutputFormat; +using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; class IVolume; using VolumePtr = std::shared_ptr; @@ -571,12 +573,9 @@ public: #endif /// I/O formats. - BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const; - - /// Don't use streams. Better look at getOutputFormat... - BlockOutputStreamPtr getOutputStreamParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const; - BlockOutputStreamPtr getOutputStream(const String & name, WriteBuffer & buf, const Block & sample) const; + InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional & format_settings = std::nullopt) const; + OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const; OutputFormatPtr getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const; InterserverIOHandler & getInterserverIOHandler(); diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index a7685f449c2..d6ad2666ff1 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -100,8 +100,8 @@ public: writeChar('\n', out); } - auto output_stream = context->getOutputStream(configuration.format, out, arguments_block.cloneEmpty()); - formatBlock(output_stream, arguments_block); + auto output_format = context->getOutputFormat(configuration.format, out, arguments_block.cloneEmpty()); + formatBlock(output_format, arguments_block); if (!is_executable_pool_function) out.close(); }}; diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index a305753d03f..8d86e896a1a 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -112,9 +112,9 @@ void ArrowBlockInputFormat::prepareReader() record_batch_current = 0; } -void registerInputFormatProcessorArrow(FormatFactory & factory) +void registerInputFormatArrow(FormatFactory & factory) { - factory.registerInputFormatProcessor( + factory.registerInputFormat( "Arrow", [](ReadBuffer & buf, const Block & sample, @@ -124,7 +124,7 @@ void registerInputFormatProcessorArrow(FormatFactory & factory) return std::make_shared(buf, sample, false, format_settings); }); factory.markFormatAsColumnOriented("Arrow"); - factory.registerInputFormatProcessor( + factory.registerInputFormat( "ArrowStream", [](ReadBuffer & buf, const Block & sample, @@ -141,7 +141,7 @@ void registerInputFormatProcessorArrow(FormatFactory & factory) namespace DB { class FormatFactory; -void registerInputFormatProcessorArrow(FormatFactory &) +void registerInputFormatArrow(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 8f43d03de38..4404749adb6 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -82,9 +82,9 @@ void ArrowBlockOutputFormat::prepareWriter(const std::shared_ptr writer = *writer_status; } -void registerOutputFormatProcessorArrow(FormatFactory & factory) +void registerOutputFormatArrow(FormatFactory & factory) { - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( "Arrow", [](WriteBuffer & buf, const Block & sample, @@ -94,7 +94,7 @@ void registerOutputFormatProcessorArrow(FormatFactory & factory) return std::make_shared(buf, sample, false, format_settings); }); - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( "ArrowStream", [](WriteBuffer & buf, const Block & sample, @@ -112,7 +112,7 @@ void registerOutputFormatProcessorArrow(FormatFactory & factory) namespace DB { class FormatFactory; -void registerOutputFormatProcessorArrow(FormatFactory &) +void registerOutputFormatArrow(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index d5aab5da74a..1f806d47c45 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -806,9 +806,9 @@ const AvroDeserializer & AvroConfluentRowInputFormat::getOrCreateDeserializer(Sc return it->second; } -void registerInputFormatProcessorAvro(FormatFactory & factory) +void registerInputFormatAvro(FormatFactory & factory) { - factory.registerInputFormatProcessor("Avro", []( + factory.registerInputFormat("Avro", []( ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, @@ -817,7 +817,7 @@ void registerInputFormatProcessorAvro(FormatFactory & factory) return std::make_shared(sample, buf, params, settings); }); - factory.registerInputFormatProcessor("AvroConfluent",[]( + factory.registerInputFormat("AvroConfluent",[]( ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, @@ -834,7 +834,7 @@ void registerInputFormatProcessorAvro(FormatFactory & factory) namespace DB { class FormatFactory; -void registerInputFormatProcessorAvro(FormatFactory &) +void registerInputFormatAvro(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 24b231e9ea8..74eeee74475 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -413,9 +413,9 @@ void AvroRowOutputFormat::writeSuffix() file_writer.close(); } -void registerOutputFormatProcessorAvro(FormatFactory & factory) +void registerOutputFormatAvro(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Avro", []( + factory.registerOutputFormat("Avro", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -432,7 +432,7 @@ void registerOutputFormatProcessorAvro(FormatFactory & factory) namespace DB { class FormatFactory; -void registerOutputFormatProcessorAvro(FormatFactory &) +void registerOutputFormatAvro(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index a7677ec7e6a..c122b9eea1a 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -56,9 +56,9 @@ void BinaryRowInputFormat::readPrefix() } -void registerInputFormatProcessorRowBinary(FormatFactory & factory) +void registerInputFormatRowBinary(FormatFactory & factory) { - factory.registerInputFormatProcessor("RowBinary", []( + factory.registerInputFormat("RowBinary", []( ReadBuffer & buf, const Block & sample, const IRowInputFormat::Params & params, @@ -67,7 +67,7 @@ void registerInputFormatProcessorRowBinary(FormatFactory & factory) return std::make_shared(buf, sample, params, false, false); }); - factory.registerInputFormatProcessor("RowBinaryWithNamesAndTypes", []( + factory.registerInputFormat("RowBinaryWithNamesAndTypes", []( ReadBuffer & buf, const Block & sample, const IRowInputFormat::Params & params, diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index 424eb375fa3..dbaee68453e 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -47,9 +47,9 @@ void BinaryRowOutputFormat::writeField(const IColumn & column, const ISerializat } -void registerOutputFormatProcessorRowBinary(FormatFactory & factory) +void registerOutputFormatRowBinary(FormatFactory & factory) { - factory.registerOutputFormatProcessor("RowBinary", []( + factory.registerOutputFormat("RowBinary", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -58,7 +58,7 @@ void registerOutputFormatProcessorRowBinary(FormatFactory & factory) return std::make_shared(buf, sample, false, false, params); }); - factory.registerOutputFormatProcessor("RowBinaryWithNamesAndTypes", []( + factory.registerOutputFormat("RowBinaryWithNamesAndTypes", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index c613288ed0a..8ccc04faf35 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -420,11 +420,11 @@ void CSVRowInputFormat::resetParser() } -void registerInputFormatProcessorCSV(FormatFactory & factory) +void registerInputFormatCSV(FormatFactory & factory) { for (bool with_names : {false, true}) { - factory.registerInputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( + factory.registerInputFormat(with_names ? "CSVWithNames" : "CSV", [=]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index b9945ddec15..14d0e519c0c 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -70,11 +70,11 @@ void CSVRowOutputFormat::writeBeforeExtremes() } -void registerOutputFormatProcessorCSV(FormatFactory & factory) +void registerOutputFormatCSV(FormatFactory & factory) { for (bool with_names : {false, true}) { - factory.registerOutputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=]( + factory.registerOutputFormat(with_names ? "CSVWithNames" : "CSV", [=]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 85397b7f6b7..fd4b2870bea 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -295,9 +295,9 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension return true; } -void registerInputFormatProcessorCapnProto(FormatFactory & factory) +void registerInputFormatCapnProto(FormatFactory & factory) { - factory.registerInputFormatProcessor( + factory.registerInputFormat( "CapnProto", [](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { @@ -314,7 +314,7 @@ void registerInputFormatProcessorCapnProto(FormatFactory & factory) namespace DB { class FormatFactory; - void registerInputFormatProcessorCapnProto(FormatFactory &) {} + void registerInputFormatCapnProto(FormatFactory &) {} } #endif // USE_CAPNP diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 10fcd55bf8c..5b32bf94c4d 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -171,9 +171,9 @@ bool JSONAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtens return !buf.eof(); } -void registerInputFormatProcessorJSONAsString(FormatFactory & factory) +void registerInputFormatJSONAsString(FormatFactory & factory) { - factory.registerInputFormatProcessor("JSONAsString", []( + factory.registerInputFormat("JSONAsString", []( ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 927fdd0bdcc..962e9d6e5ac 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -236,9 +236,9 @@ void JSONCompactEachRowRowInputFormat::syncAfterError() skipToUnescapedNextLineOrEOF(*in); } -void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) +void registerInputFormatJSONCompactEachRow(FormatFactory & factory) { - factory.registerInputFormatProcessor("JSONCompactEachRow", []( + factory.registerInputFormat("JSONCompactEachRow", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -247,7 +247,7 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) return std::make_shared(buf, sample, std::move(params), settings, false, false); }); - factory.registerInputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( + factory.registerInputFormat("JSONCompactEachRowWithNamesAndTypes", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -256,7 +256,7 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) return std::make_shared(buf, sample, std::move(params), settings, true, false); }); - factory.registerInputFormatProcessor("JSONCompactStringsEachRow", []( + factory.registerInputFormat("JSONCompactStringsEachRow", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -265,7 +265,7 @@ void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) return std::make_shared(buf, sample, std::move(params), settings, false, true); }); - factory.registerInputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + factory.registerInputFormat("JSONCompactStringsEachRowWithNamesAndTypes", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index a3055873c01..1ce4277023d 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -98,9 +98,9 @@ void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) IRowOutputFormat::consumeTotals(std::move(chunk)); } -void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) +void registerOutputFormatJSONCompactEachRow(FormatFactory & factory) { - factory.registerOutputFormatProcessor("JSONCompactEachRow", []( + factory.registerOutputFormat("JSONCompactEachRow", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -110,7 +110,7 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONCompactEachRow"); - factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( + factory.registerOutputFormat("JSONCompactEachRowWithNamesAndTypes", []( WriteBuffer &buf, const Block &sample, const RowOutputFormatParams & params, @@ -120,7 +120,7 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONCompactEachRowWithNamesAndTypes"); - factory.registerOutputFormatProcessor("JSONCompactStringsEachRow", []( + factory.registerOutputFormat("JSONCompactStringsEachRow", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -130,7 +130,7 @@ void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONCompactStringsEachRow"); - factory.registerOutputFormatProcessor("JSONCompactStringsEachRowWithNamesAndTypes", []( + factory.registerOutputFormat("JSONCompactStringsEachRowWithNamesAndTypes", []( WriteBuffer &buf, const Block &sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp index cefaded6912..f4887b66968 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.cpp @@ -88,9 +88,9 @@ void JSONCompactRowOutputFormat::writeExtremesElement(const char * title, const writeChar(']', *ostr); } -void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) +void registerOutputFormatJSONCompact(FormatFactory & factory) { - factory.registerOutputFormatProcessor("JSONCompact", []( + factory.registerOutputFormat("JSONCompact", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -99,7 +99,7 @@ void registerOutputFormatProcessorJSONCompact(FormatFactory & factory) return std::make_shared(buf, sample, params, format_settings, false); }); - factory.registerOutputFormatProcessor("JSONCompactStrings", []( + factory.registerOutputFormat("JSONCompactStrings", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 9f3ab8747ed..d04ba2a49e4 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -332,9 +332,9 @@ void JSONEachRowRowInputFormat::readSuffix() } -void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) +void registerInputFormatJSONEachRow(FormatFactory & factory) { - factory.registerInputFormatProcessor("JSONEachRow", []( + factory.registerInputFormat("JSONEachRow", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -343,7 +343,7 @@ void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) return std::make_shared(buf, sample, std::move(params), settings, false); }); - factory.registerInputFormatProcessor("JSONStringsEachRow", []( + factory.registerInputFormat("JSONStringsEachRow", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp index a69499de813..0a07da1858b 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.cpp @@ -125,9 +125,9 @@ void JSONEachRowRowOutputFormat::writeSuffix() } -void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) +void registerOutputFormatJSONEachRow(FormatFactory & factory) { - factory.registerOutputFormatProcessor("JSONEachRow", []( + factory.registerOutputFormat("JSONEachRow", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -140,7 +140,7 @@ void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONEachRow"); - factory.registerOutputFormatProcessor("JSONStringsEachRow", []( + factory.registerOutputFormat("JSONStringsEachRow", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp index 4612ce99f05..b9660a5bc00 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -28,9 +28,9 @@ void JSONEachRowWithProgressRowOutputFormat::onProgress(const Progress & value) } -void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory) +void registerOutputFormatJSONEachRowWithProgress(FormatFactory & factory) { - factory.registerOutputFormatProcessor("JSONEachRowWithProgress", []( + factory.registerOutputFormat("JSONEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -42,7 +42,7 @@ void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factor sample, params, settings); }); - factory.registerOutputFormatProcessor("JSONStringsEachRowWithProgress", []( + factory.registerOutputFormat("JSONStringsEachRowWithProgress", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 38c6eefac1c..28d774fea74 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -266,9 +266,9 @@ void JSONRowOutputFormat::onProgress(const Progress & value) } -void registerOutputFormatProcessorJSON(FormatFactory & factory) +void registerOutputFormatJSON(FormatFactory & factory) { - factory.registerOutputFormatProcessor("JSON", []( + factory.registerOutputFormat("JSON", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -277,7 +277,7 @@ void registerOutputFormatProcessorJSON(FormatFactory & factory) return std::make_shared(buf, sample, params, format_settings, false); }); - factory.registerOutputFormatProcessor("JSONStrings", []( + factory.registerOutputFormat("JSONStrings", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index a062d331a13..1a05f61d36b 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -61,9 +61,9 @@ bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtens return true; } -void registerInputFormatProcessorLineAsString(FormatFactory & factory) +void registerInputFormatLineAsString(FormatFactory & factory) { - factory.registerInputFormatProcessor("LineAsString", []( + factory.registerInputFormat("LineAsString", []( ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp index ebf0384445c..40fe5a77d62 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.cpp @@ -55,9 +55,9 @@ void MarkdownRowOutputFormat::writeField(const IColumn & column, const ISerializ serialization.serializeTextEscaped(column, row_num, out, format_settings); } -void registerOutputFormatProcessorMarkdown(FormatFactory & factory) +void registerOutputFormatMarkdown(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Markdown", []( + factory.registerOutputFormat("Markdown", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 2b568166d5b..86540be17b9 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -363,9 +363,9 @@ bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & return true; } -void registerInputFormatProcessorMsgPack(FormatFactory & factory) +void registerInputFormatMsgPack(FormatFactory & factory) { - factory.registerInputFormatProcessor("MsgPack", []( + factory.registerInputFormat("MsgPack", []( ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, @@ -382,7 +382,7 @@ void registerInputFormatProcessorMsgPack(FormatFactory & factory) namespace DB { class FormatFactory; -void registerInputFormatProcessorMsgPack(FormatFactory &) +void registerInputFormatMsgPack(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index 27f736128f7..bbc7b005fd8 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -180,10 +180,10 @@ void MsgPackRowOutputFormat::write(const Columns & columns, size_t row_num) } -void registerOutputFormatProcessorMsgPack(FormatFactory & factory) +void registerOutputFormatMsgPack(FormatFactory & factory) { - factory.registerOutputFormatProcessor("MsgPack", []( + factory.registerOutputFormat("MsgPack", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -200,7 +200,7 @@ void registerOutputFormatProcessorMsgPack(FormatFactory & factory) namespace DB { class FormatFactory; -void registerOutputFormatProcessorMsgPack(FormatFactory &) +void registerOutputFormatMsgPack(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 4c6f4624ad4..0b366244611 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -105,9 +105,9 @@ void MySQLOutputFormat::flush() packet_endpoint->out->next(); } -void registerOutputFormatProcessorMySQLWire(FormatFactory & factory) +void registerOutputFormatMySQLWire(FormatFactory & factory) { - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( "MySQLWire", [](WriteBuffer & buf, const Block & sample, diff --git a/src/Processors/Formats/Impl/NullFormat.cpp b/src/Processors/Formats/Impl/NullFormat.cpp index fda79d6981a..a22005a1c12 100644 --- a/src/Processors/Formats/Impl/NullFormat.cpp +++ b/src/Processors/Formats/Impl/NullFormat.cpp @@ -16,9 +16,9 @@ protected: void consume(Chunk) override {} }; -void registerOutputFormatProcessorNull(FormatFactory & factory) +void registerOutputFormatNull(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Null", []( + factory.registerOutputFormat("Null", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, diff --git a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index 7a14966e220..0e486715c98 100644 --- a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -110,9 +110,9 @@ void ODBCDriver2BlockOutputFormat::writePrefix() } -void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory) +void registerOutputFormatODBCDriver2(FormatFactory & factory) { - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, const FormatSettings & format_settings) { return std::make_shared(buf, sample, format_settings); diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index aa158788394..d9866edbccf 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -124,9 +124,9 @@ void ORCBlockInputFormat::prepareReader() } } -void registerInputFormatProcessorORC(FormatFactory &factory) +void registerInputFormatORC(FormatFactory &factory) { - factory.registerInputFormatProcessor( + factory.registerInputFormat( "ORC", [](ReadBuffer &buf, const Block &sample, @@ -144,7 +144,7 @@ void registerInputFormatProcessorORC(FormatFactory &factory) namespace DB { class FormatFactory; - void registerInputFormatProcessorORC(FormatFactory &) + void registerInputFormatORC(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 78c8b324689..0a1eb9c6836 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -514,9 +514,9 @@ void ORCBlockOutputFormat::prepareWriter() writer = orc::createWriter(*schema, &output_stream, options); } -void registerOutputFormatProcessorORC(FormatFactory & factory) +void registerOutputFormatORC(FormatFactory & factory) { - factory.registerOutputFormatProcessor("ORC", []( + factory.registerOutputFormat("ORC", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, @@ -533,7 +533,7 @@ void registerOutputFormatProcessorORC(FormatFactory & factory) namespace DB { class FormatFactory; - void registerOutputFormatProcessorORC(FormatFactory &) + void registerOutputFormatORC(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 8f088a3f84a..51be1f13997 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -123,9 +123,9 @@ void ParquetBlockInputFormat::prepareReader() } } -void registerInputFormatProcessorParquet(FormatFactory &factory) +void registerInputFormatParquet(FormatFactory &factory) { - factory.registerInputFormatProcessor( + factory.registerInputFormat( "Parquet", [](ReadBuffer &buf, const Block &sample, @@ -144,7 +144,7 @@ void registerInputFormatProcessorParquet(FormatFactory &factory) namespace DB { class FormatFactory; -void registerInputFormatProcessorParquet(FormatFactory &) +void registerInputFormatParquet(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index c3771c7b552..416d1b80988 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -74,9 +74,9 @@ void ParquetBlockOutputFormat::finalize() throw Exception{"Error while closing a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; } -void registerOutputFormatProcessorParquet(FormatFactory & factory) +void registerOutputFormatParquet(FormatFactory & factory) { - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( "Parquet", [](WriteBuffer & buf, const Block & sample, @@ -94,7 +94,7 @@ void registerOutputFormatProcessorParquet(FormatFactory & factory) namespace DB { class FormatFactory; -void registerOutputFormatProcessorParquet(FormatFactory &) +void registerOutputFormatParquet(FormatFactory &) { } } diff --git a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp index 8c4da279fc5..a5f92e41da4 100644 --- a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp @@ -68,9 +68,9 @@ void PostgreSQLOutputFormat::flush() message_transport.flush(); } -void registerOutputFormatProcessorPostgreSQLWire(FormatFactory & factory) +void registerOutputFormatPostgreSQLWire(FormatFactory & factory) { - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( "PostgreSQLWire", [](WriteBuffer & buf, const Block & sample, diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 8e178b6629e..2bae0c6bd63 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -407,9 +407,9 @@ void PrettyBlockOutputFormat::finalize() } -void registerOutputFormatProcessorPretty(FormatFactory & factory) +void registerOutputFormatPretty(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Pretty", []( + factory.registerOutputFormat("Pretty", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, @@ -418,7 +418,7 @@ void registerOutputFormatProcessorPretty(FormatFactory & factory) return std::make_shared(buf, sample, format_settings); }); - factory.registerOutputFormatProcessor("PrettyNoEscapes", []( + factory.registerOutputFormat("PrettyNoEscapes", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index c4902ea4c26..9ae7ccbd240 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -255,11 +255,11 @@ void PrettyCompactBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind po } -void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) +void registerOutputFormatPrettyCompact(FormatFactory & factory) { for (const auto & [name, mono_block] : {std::make_pair("PrettyCompact", false), std::make_pair("PrettyCompactMonoBlock", true)}) { - factory.registerOutputFormatProcessor(name, [mono_block = mono_block]( + factory.registerOutputFormat(name, [mono_block = mono_block]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, @@ -269,7 +269,7 @@ void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory) }); } - factory.registerOutputFormatProcessor("PrettyCompactNoEscapes", []( + factory.registerOutputFormat("PrettyCompactNoEscapes", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index fa987c6b949..d317a21b769 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -113,9 +113,9 @@ void PrettySpaceBlockOutputFormat::writeSuffix() } -void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) +void registerOutputFormatPrettySpace(FormatFactory & factory) { - factory.registerOutputFormatProcessor("PrettySpace", []( + factory.registerOutputFormat("PrettySpace", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, @@ -124,7 +124,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory) return std::make_shared(buf, sample, format_settings); }); - factory.registerOutputFormatProcessor("PrettySpaceNoEscapes", []( + factory.registerOutputFormat("PrettySpaceNoEscapes", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index f7b8c144e0d..a5e6b7ec480 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -56,11 +56,11 @@ void ProtobufRowInputFormat::syncAfterError() reader->endMessage(true); } -void registerInputFormatProcessorProtobuf(FormatFactory & factory) +void registerInputFormatProtobuf(FormatFactory & factory) { for (bool with_length_delimiter : {false, true}) { - factory.registerInputFormatProcessor(with_length_delimiter ? "Protobuf" : "ProtobufSingle", [with_length_delimiter]( + factory.registerInputFormat(with_length_delimiter ? "Protobuf" : "ProtobufSingle", [with_length_delimiter]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -81,7 +81,7 @@ void registerInputFormatProcessorProtobuf(FormatFactory & factory) namespace DB { class FormatFactory; -void registerInputFormatProcessorProtobuf(FormatFactory &) {} +void registerInputFormatProtobuf(FormatFactory &) {} } #endif diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index d3b9a0124c1..12c5e98797a 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -51,11 +51,11 @@ void ProtobufRowOutputFormat::write(const Columns & columns, size_t row_num) } -void registerOutputFormatProcessorProtobuf(FormatFactory & factory) +void registerOutputFormatProtobuf(FormatFactory & factory) { for (bool with_length_delimiter : {false, true}) { - factory.registerOutputFormatProcessor( + factory.registerOutputFormat( with_length_delimiter ? "Protobuf" : "ProtobufSingle", [with_length_delimiter](WriteBuffer & buf, const Block & header, @@ -80,7 +80,7 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory) namespace DB { class FormatFactory; - void registerOutputFormatProcessorProtobuf(FormatFactory &) {} + void registerOutputFormatProtobuf(FormatFactory &) {} } #endif diff --git a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp index 3b1317acc63..34424fffd34 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp @@ -39,9 +39,9 @@ bool RawBLOBRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & return false; } -void registerInputFormatProcessorRawBLOB(FormatFactory & factory) +void registerInputFormatRawBLOB(FormatFactory & factory) { - factory.registerInputFormatProcessor("RawBLOB", []( + factory.registerInputFormat("RawBLOB", []( ReadBuffer & buf, const Block & sample, const RowInputFormatParams & params, diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp index 49f1159d48d..9a38a11f6f0 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp @@ -22,9 +22,9 @@ void RawBLOBRowOutputFormat::writeField(const IColumn & column, const ISerializa } -void registerOutputFormatProcessorRawBLOB(FormatFactory & factory) +void registerOutputFormatRawBLOB(FormatFactory & factory) { - factory.registerOutputFormatProcessor("RawBLOB", []( + factory.registerOutputFormat("RawBLOB", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index c44dd912562..8cb0fce609e 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -163,9 +163,9 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & return true; } -void registerInputFormatProcessorRegexp(FormatFactory & factory) +void registerInputFormatRegexp(FormatFactory & factory) { - factory.registerInputFormatProcessor("Regexp", []( + factory.registerInputFormat("Regexp", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index df992a965b1..331d6e435d1 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -210,9 +210,9 @@ void TSKVRowInputFormat::resetParser() name_buf.clear(); } -void registerInputFormatProcessorTSKV(FormatFactory & factory) +void registerInputFormatTSKV(FormatFactory & factory) { - factory.registerInputFormatProcessor("TSKV", []( + factory.registerInputFormat("TSKV", []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index 627ae67fa31..6161303d23a 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -39,9 +39,9 @@ void TSKVRowOutputFormat::writeRowEndDelimiter() } -void registerOutputFormatProcessorTSKV(FormatFactory & factory) +void registerOutputFormatTSKV(FormatFactory & factory) { - factory.registerOutputFormatProcessor("TSKV", []( + factory.registerOutputFormat("TSKV", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 7fe23a5bb94..1ff52c9f695 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -383,11 +383,11 @@ void TabSeparatedRowInputFormat::resetParser() columns_to_fill_with_default_values.clear(); } -void registerInputFormatProcessorTabSeparated(FormatFactory & factory) +void registerInputFormatTabSeparated(FormatFactory & factory) { for (const auto * name : {"TabSeparated", "TSV"}) { - factory.registerInputFormatProcessor(name, []( + factory.registerInputFormat(name, []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -399,7 +399,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) for (const auto * name : {"TabSeparatedRaw", "TSVRaw"}) { - factory.registerInputFormatProcessor(name, []( + factory.registerInputFormat(name, []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -411,7 +411,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) for (const auto * name : {"TabSeparatedWithNames", "TSVWithNames"}) { - factory.registerInputFormatProcessor(name, []( + factory.registerInputFormat(name, []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -423,7 +423,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) for (const auto * name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) { - factory.registerInputFormatProcessor(name, []( + factory.registerInputFormat(name, []( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 3e99264785e..71d5bdba355 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -73,11 +73,11 @@ void TabSeparatedRowOutputFormat::writeBeforeExtremes() } -void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) +void registerOutputFormatTabSeparated(FormatFactory & factory) { for (const auto * name : {"TabSeparated", "TSV"}) { - factory.registerOutputFormatProcessor(name, []( + factory.registerOutputFormat(name, []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -90,7 +90,7 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) for (const auto * name : {"TabSeparatedRaw", "TSVRaw"}) { - factory.registerOutputFormatProcessor(name, []( + factory.registerOutputFormat(name, []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -103,7 +103,7 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) for (const auto * name : {"TabSeparatedWithNames", "TSVWithNames"}) { - factory.registerOutputFormatProcessor(name, []( + factory.registerOutputFormat(name, []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, @@ -116,7 +116,7 @@ void registerOutputFormatProcessorTabSeparated(FormatFactory & factory) for (const auto * name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) { - factory.registerOutputFormatProcessor(name, []( + factory.registerOutputFormat(name, []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index d65f6dd9e38..ed98ab372b6 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -227,9 +227,9 @@ void TemplateBlockOutputFormat::finalize() } -void registerOutputFormatProcessorTemplate(FormatFactory & factory) +void registerOutputFormatTemplate(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Template", []( + factory.registerOutputFormat("Template", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, @@ -267,7 +267,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory & factory) return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); - factory.registerOutputFormatProcessor("CustomSeparated", []( + factory.registerOutputFormat("CustomSeparated", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams &, diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index 0e5a962a037..021b2532b39 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -503,11 +503,11 @@ void TemplateRowInputFormat::resetParser() buf.reset(); } -void registerInputFormatProcessorTemplate(FormatFactory & factory) +void registerInputFormatTemplate(FormatFactory & factory) { for (bool ignore_spaces : {false, true}) { - factory.registerInputFormatProcessor(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=]( + factory.registerInputFormat(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, @@ -551,7 +551,7 @@ void registerInputFormatProcessorTemplate(FormatFactory & factory) for (bool ignore_spaces : {false, true}) { - factory.registerInputFormatProcessor(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", [=]( + factory.registerInputFormat(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", [=]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 6584c99aac7..0f6a21055d0 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -543,9 +543,9 @@ void ValuesBlockInputFormat::resetParser() total_rows = 0; } -void registerInputFormatProcessorValues(FormatFactory & factory) +void registerInputFormatValues(FormatFactory & factory) { - factory.registerInputFormatProcessor("Values", []( + factory.registerInputFormat("Values", []( ReadBuffer & buf, const Block & header, const RowInputFormatParams & params, diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp b/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp index e0152a7ffee..f63483c1f90 100644 --- a/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesRowOutputFormat.cpp @@ -41,9 +41,9 @@ void ValuesRowOutputFormat::writeRowBetweenDelimiter() } -void registerOutputFormatProcessorValues(FormatFactory & factory) +void registerOutputFormatValues(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Values", []( + factory.registerOutputFormat("Values", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp b/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp index c6f37d270b0..c30cb2ccbbb 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp @@ -161,9 +161,9 @@ void VerticalRowOutputFormat::writeSpecialRow(const Columns & columns, size_t ro } } -void registerOutputFormatProcessorVertical(FormatFactory & factory) +void registerOutputFormatVertical(FormatFactory & factory) { - factory.registerOutputFormatProcessor("Vertical", []( + factory.registerOutputFormat("Vertical", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 893c4e229c7..8ddc2ca4bbe 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -240,9 +240,9 @@ void XMLRowOutputFormat::writeStatistics() } -void registerOutputFormatProcessorXML(FormatFactory & factory) +void registerOutputFormatXML(FormatFactory & factory) { - factory.registerOutputFormatProcessor("XML", []( + factory.registerOutputFormat("XML", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 3f8328ac835..a1cce03a1a5 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include @@ -60,7 +59,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( : std::make_unique(); /// Create a source from input buffer using format from query - auto source = FormatFactory::instance().getInput(format, *input_buffer, header, context, context->getSettings().max_insert_block_size); + auto source = context->getInputFormat(format, *input_buffer, header, context->getSettings().max_insert_block_size); source->addBuffer(std::move(input_buffer)); return source; } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index cc3c7085dfd..0ee9727f3d6 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -27,10 +27,10 @@ #include #include #include +#include #include #include #include -#include #include #include #include @@ -634,7 +634,7 @@ namespace std::optional write_buffer; std::unique_ptr pipeline; std::unique_ptr pipeline_executor; - BlockOutputStreamPtr block_output_stream; + std::shared_ptr output_format_processor; bool need_input_data_from_insert_query = true; bool need_input_data_from_query_info = true; bool need_input_data_delimiter = false; @@ -943,8 +943,8 @@ namespace }); assert(!pipeline); - auto source = FormatFactory::instance().getInput( - input_format, *read_buffer, header, query_context, query_context->getSettings().max_insert_block_size); + auto source = query_context->getInputFormat( + input_format, *read_buffer, header, query_context->getSettings().max_insert_block_size); QueryPipelineBuilder builder; builder.init(Pipe(source)); @@ -1030,9 +1030,9 @@ namespace external_table_context->checkSettingsConstraints(settings_changes); external_table_context->applySettingsChanges(settings_changes); } - auto in = FormatFactory::instance().getInput( + auto in = external_table_context->getInputFormat( format, data, metadata_snapshot->getSampleBlock(), - external_table_context, external_table_context->getSettings().max_insert_block_size); + external_table_context->getSettings().max_insert_block_size); QueryPipelineBuilder cur_pipeline; cur_pipeline.init(Pipe(std::move(in))); @@ -1086,8 +1086,8 @@ namespace header = io.pipeline.getHeader(); write_buffer.emplace(*result.mutable_output()); - block_output_stream = query_context->getOutputStream(output_format, *write_buffer, header); - block_output_stream->writePrefix(); + output_format_processor = query_context->getOutputFormat(output_format, *write_buffer, header); + output_format_processor->doWritePrefix(); Stopwatch after_send_progress; /// Unless the input() function is used we are not going to receive input data anymore. @@ -1118,7 +1118,7 @@ namespace break; if (block && !io.null_format) - block_output_stream->write(block); + output_format_processor->write(block); if (after_send_progress.elapsedMicroseconds() >= interactive_delay) { @@ -1166,7 +1166,7 @@ namespace executor->execute(); } - block_output_stream->writeSuffix(); + output_format_processor->doWriteSuffix(); } void Call::finishQuery() @@ -1240,7 +1240,7 @@ namespace responder.reset(); pipeline_executor.reset(); pipeline.reset(); - block_output_stream.reset(); + output_format_processor.reset(); read_buffer.reset(); write_buffer.reset(); io = {}; @@ -1362,10 +1362,10 @@ namespace return; WriteBufferFromString buf{*result.mutable_totals()}; - auto stream = query_context->getOutputStream(output_format, buf, totals); - stream->writePrefix(); - stream->write(totals); - stream->writeSuffix(); + auto format = query_context->getOutputFormat(output_format, buf, totals); + format->doWritePrefix(); + format->write(totals); + format->doWriteSuffix(); } void Call::addExtremesToResult(const Block & extremes) @@ -1374,10 +1374,10 @@ namespace return; WriteBufferFromString buf{*result.mutable_extremes()}; - auto stream = query_context->getOutputStream(output_format, buf, extremes); - stream->writePrefix(); - stream->write(extremes); - stream->writeSuffix(); + auto format = query_context->getOutputFormat(output_format, buf, extremes); + format->doWritePrefix(); + format->write(extremes); + format->doWriteSuffix(); } void Call::addProfileInfoToResult(const BlockStreamProfileInfo & info) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 3e6b9da211f..19385e526a7 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -122,7 +123,7 @@ public: auto compression = chooseCompressionMethod(path, compression_method); read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri, path, getContext()->getGlobalContext()->getConfigRef()), compression); - auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size); + auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size); reader = std::make_shared(input_format); reader->readPrefix(); @@ -182,7 +183,7 @@ public: : SinkToStorage(sample_block) { write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef()), compression_method, 3); - writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, *write_buf, sample_block, context); + writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } String getName() const override { return "HDFSSink"; } @@ -191,7 +192,7 @@ public: { if (is_first_chunk) { - writer->writePrefix(); + writer->doWritePrefix(); is_first_chunk = false; } writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); @@ -201,7 +202,7 @@ public: { try { - writer->writeSuffix(); + writer->doWriteSuffix(); writer->flush(); write_buf->sync(); write_buf->finalize(); @@ -215,7 +216,7 @@ public: private: std::unique_ptr write_buf; - BlockOutputStreamPtr writer; + OutputFormatPtr writer; bool is_first_chunk = true; }; diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 395065de084..d3c51fef9b7 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -1,6 +1,7 @@ #include #include +#include #include namespace DB @@ -24,7 +25,7 @@ void KafkaSink::onStart() auto format_settings = getFormatSettings(context); format_settings.protobuf.allow_multiple_rows_without_delimiter = true; - child = FormatFactory::instance().getOutputStream(storage.getFormatName(), *buffer, + format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & columns, size_t row) { @@ -35,13 +36,13 @@ void KafkaSink::onStart() void KafkaSink::consume(Chunk chunk) { - child->write(getHeader().cloneWithColumns(chunk.detachColumns())); + format->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void KafkaSink::onFinish() { - if (child) - child->writeSuffix(); + if (format) + format->doWriteSuffix(); //flush(); if (buffer) diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.h b/src/Storages/Kafka/KafkaBlockOutputStream.h index 5529cd8ece8..8e0801c7b5e 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.h +++ b/src/Storages/Kafka/KafkaBlockOutputStream.h @@ -6,6 +6,9 @@ namespace DB { +class IOutputFormat; +using IOutputFormatPtr = std::shared_ptr; + class KafkaSink : public SinkToStorage { public: @@ -26,7 +29,7 @@ private: StorageMetadataPtr metadata_snapshot; const ContextPtr context; ProducerBufferPtr buffer; - BlockOutputStreamPtr child; + IOutputFormatPtr format; }; } diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 68fc17a97e5..926432395e2 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -78,8 +78,8 @@ Chunk KafkaSource::generateImpl() auto put_error_to_stream = handle_error_mode == HandleKafkaErrorMode::STREAM; - auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + auto input_format = context->getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, max_block_size); std::optional exception_message; size_t total_rows = 0; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c04e0d2e38f..683689a6fa3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -3477,11 +3476,10 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc buf.appendBuffer(std::make_unique(partition_ast.fields_str.data(), partition_ast.fields_str.size())); buf.appendBuffer(std::make_unique(")", 1)); - auto input_format = FormatFactory::instance().getInput( + auto input_format = local_context->getInputFormat( "Values", buf, metadata_snapshot->getPartitionKey().sample_block, - local_context, local_context->getSettingsRef().max_block_size); auto input_stream = std::make_shared(input_format); diff --git a/src/Storages/RabbitMQ/RabbitMQSink.cpp b/src/Storages/RabbitMQ/RabbitMQSink.cpp index bece0d028ec..2b8d5ab3810 100644 --- a/src/Storages/RabbitMQ/RabbitMQSink.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSink.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -31,7 +32,7 @@ void RabbitMQSink::onStart() auto format_settings = getFormatSettings(context); format_settings.protobuf.allow_multiple_rows_without_delimiter = true; - child = FormatFactory::instance().getOutputStream(storage.getFormatName(), *buffer, getHeader(), context, + format = FormatFactory::instance().getOutputFormat(storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & /* columns */, size_t /* rows */) { buffer->countRow(); @@ -42,13 +43,13 @@ void RabbitMQSink::onStart() void RabbitMQSink::consume(Chunk chunk) { - child->write(getHeader().cloneWithColumns(chunk.detachColumns())); + format->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void RabbitMQSink::onFinish() { - child->writeSuffix(); + format->doWriteSuffix(); if (buffer) buffer->updateMaxWait(); diff --git a/src/Storages/RabbitMQ/RabbitMQSink.h b/src/Storages/RabbitMQ/RabbitMQSink.h index 6222ccdf2ac..02014a3f89e 100644 --- a/src/Storages/RabbitMQ/RabbitMQSink.h +++ b/src/Storages/RabbitMQ/RabbitMQSink.h @@ -7,6 +7,9 @@ namespace DB { +class IOutputFormat; +using IOutputFormatPtr = std::shared_ptr; + class RabbitMQSink : public SinkToStorage { @@ -24,6 +27,6 @@ private: StorageMetadataPtr metadata_snapshot; ContextPtr context; ProducerBufferPtr buffer; - BlockOutputStreamPtr child; + IOutputFormatPtr format; }; } diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 34edd06d3e2..2ea60bfb68b 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -119,8 +118,8 @@ Chunk RabbitMQSource::generateImpl() is_finished = true; MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - auto input_format = FormatFactory::instance().getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + auto input_format = context->getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, max_block_size); StreamingFormatExecutor executor(non_virtual_header, input_format); diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 989cc3a1f91..d6e242d1a97 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -186,7 +185,7 @@ Pipe StorageExecutable::read( auto pipeline = std::make_shared(QueryPipelineBuilder::getPipeline(std::move(inputs[i]))); - auto out = FormatFactory::instance().getOutputFormat(format, *write_buffer, materializeBlock(pipeline->getHeader()), context); + auto out = context->getOutputFormat(format, *write_buffer, materializeBlock(pipeline->getHeader())); out->setAutoFlush(); pipeline->complete(std::move(out)); diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index ce76ab4a723..cedb7d26b48 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c6c40453874..ec0bd5e5840 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -390,8 +391,8 @@ public: return metadata_snapshot->getSampleBlock(); }; - auto format = FormatFactory::instance().getInput( - storage->format_name, *read_buf, get_block_for_format(), context, max_block_size, storage->format_settings); + auto format = context->getInputFormat( + storage->format_name, *read_buf, get_block_for_format(), max_block_size, storage->format_settings); QueryPipelineBuilder builder; builder.init(Pipe(format)); @@ -574,7 +575,7 @@ public: write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); - writer = FormatFactory::instance().getOutputStreamParallelIfPossible(storage.format_name, + writer = FormatFactory::instance().getOutputFormatParallelIfPossible(storage.format_name, *write_buf, metadata_snapshot->getSampleBlock(), context, {}, format_settings); } @@ -584,7 +585,7 @@ public: void onStart() override { if (!prefix_written) - writer->writePrefix(); + writer->doWritePrefix(); prefix_written = true; } @@ -595,7 +596,7 @@ public: void onFinish() override { - writer->writeSuffix(); + writer->doWriteSuffix(); } // void flush() override @@ -608,7 +609,7 @@ private: StorageMetadataPtr metadata_snapshot; std::unique_lock lock; std::unique_ptr write_buf; - BlockOutputStreamPtr writer; + OutputFormatPtr writer; bool prefix_written{false}; }; diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index f2907c0d707..001684c076d 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -162,7 +163,7 @@ public: sqlbuf << backQuoteMySQL(remote_table_name); sqlbuf << " (" << dumpNamesWithBackQuote(block) << ") VALUES "; - auto writer = FormatFactory::instance().getOutputStream("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext()); + auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext()); writer->write(block); if (!storage.on_duplicate_clause.empty()) diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 1c2d5418ef7..96e72c77f00 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -4,7 +4,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index edb4713a60e..7949cac6249 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -22,7 +22,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ce1b049758e..caefa097c3a 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -32,6 +32,7 @@ #include #include +#include #include #include @@ -74,6 +75,10 @@ namespace ErrorCodes extern const int S3_ERROR; extern const int UNEXPECTED_EXPRESSION; } + +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + class StorageS3Source::DisclosedGlobIterator::Impl { @@ -232,7 +237,7 @@ bool StorageS3Source::initialize() read_buf = wrapReadBufferWithCompressionMethod( std::make_unique(client, bucket, current_key, max_single_read_retries, DBMS_DEFAULT_BUFFER_SIZE), chooseCompressionMethod(current_key, compression_hint)); - auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size, format_settings); + auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size, format_settings); QueryPipelineBuilder builder; builder.init(Pipe(input_format)); @@ -309,7 +314,7 @@ public: { write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3); - writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings); + writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings); } String getName() const override { return "StorageS3Sink"; } @@ -318,7 +323,7 @@ public: { if (is_first_chunk) { - writer->writePrefix(); + writer->doWritePrefix(); is_first_chunk = false; } writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); @@ -328,7 +333,7 @@ public: { try { - writer->writeSuffix(); + writer->doWriteSuffix(); writer->flush(); write_buf->finalize(); } @@ -344,7 +349,7 @@ private: Block sample_block; std::optional format_settings; std::unique_ptr write_buf; - BlockOutputStreamPtr writer; + OutputFormatPtr writer; bool is_first_chunk = true; }; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index c9a0a4873c3..e4682efeaad 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 9a37696bb1c..32660cb1b1f 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -121,7 +122,7 @@ public: sqlbuf << ") VALUES "; - auto writer = FormatFactory::instance().getOutputStream("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext()); + auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext()); writer->write(block); sqlbuf << ";"; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cbd935f7909..75ad2761362 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -12,6 +12,7 @@ #include #include +#include #include #include @@ -204,7 +205,7 @@ StorageURLSink::StorageURLSink( write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts), compression_method, 3); - writer = FormatFactory::instance().getOutputStream(format, *write_buf, sample_block, + writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, {} /* write callback */, format_settings); } @@ -213,7 +214,7 @@ void StorageURLSink::consume(Chunk chunk) { if (is_first_chunk) { - writer->writePrefix(); + writer->doWritePrefix(); is_first_chunk = false; } @@ -222,7 +223,7 @@ void StorageURLSink::consume(Chunk chunk) void StorageURLSink::onFinish() { - writer->writeSuffix(); + writer->doWriteSuffix(); writer->flush(); write_buf->finalize(); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fdff2cff077..1f2cac239e1 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -14,6 +14,9 @@ namespace DB { +class IOutputFormat; +using OutputFormatPtr = std::shared_ptr; + struct ConnectionTimeouts; /** @@ -99,7 +102,7 @@ public: private: std::unique_ptr write_buf; - BlockOutputStreamPtr writer; + OutputFormatPtr writer; bool is_first_chunk = true; }; diff --git a/src/Storages/System/StorageSystemFormats.cpp b/src/Storages/System/StorageSystemFormats.cpp index 86e0212a523..b35fdd3f85e 100644 --- a/src/Storages/System/StorageSystemFormats.cpp +++ b/src/Storages/System/StorageSystemFormats.cpp @@ -21,8 +21,8 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns, ContextPtr, co for (const auto & pair : formats) { const auto & [format_name, creators] = pair; - UInt64 has_input_format(creators.input_creator != nullptr || creators.input_processor_creator != nullptr); - UInt64 has_output_format(creators.output_creator != nullptr || creators.output_processor_creator != nullptr); + UInt64 has_input_format(creators.input_creator != nullptr); + UInt64 has_output_format(creators.output_creator != nullptr); res_columns[0]->insert(format_name); res_columns[1]->insert(has_input_format); res_columns[2]->insert(has_output_format); diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 6cd03cad6d0..e7ecfc7c4f0 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -125,7 +126,6 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context) context, QueryProcessingStage::Complete, metadata_snapshot, query_info); QueryPipeline pipeline(table->read(column_names, metadata_snapshot, query_info, context, stage, 8192, 1)); - PullingPipelineExecutor executor(pipeline); Block sample; { @@ -138,16 +138,16 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context) tryRegisterFormats(); WriteBufferFromOwnString out_buf; - BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream("Values", out_buf, sample, context); + auto output = FormatFactory::instance().getOutputFormat("Values", out_buf, sample, context); + pipeline.complete(output); Block data; - output->writePrefix(); - while (executor.pull(data)) - output->write(data); - output->writeSuffix(); - output->flush(); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + // output->flush(); + out_buf.finalize(); return out_buf.str(); } diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index f732e160751..dcb91c8cc2e 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -2,7 +2,6 @@ #include "registerTableFunctions.h" #include -#include #include #include #include diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 4b78862a269..bbaa1b5f048 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -2,7 +2,6 @@ #include "registerTableFunctions.h" #include -#include #include #include #include From a3d629a5b541ef2d0489b9b7e7e710ed3c7a0410 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Mon, 11 Oct 2021 22:51:13 +0530 Subject: [PATCH 454/950] add x86 feature avx2/avx512 support for filter implementation --- cmake/cpu_features.cmake | 32 ++++++++++++ src/Columns/ColumnFixedString.cpp | 69 ++++++++++++++++++++++++- src/Columns/ColumnVector.cpp | 63 +++++++++++++++++++++- src/Columns/ColumnsCommon.cpp | 86 ++++++++++++++++++++++++++++++- 4 files changed, 246 insertions(+), 4 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 46e42329958..d77ca0b32e3 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,6 +18,8 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) +option (ENABLE_BMI "Use BMI instructions on x86_64" 1) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -127,6 +129,36 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () + + set (TEST_FLAG "-mavx512f -mavx512bw") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm512_setzero_epi32(); + (void)a; + auto b = _mm512_add_epi16(__m512i(), __m512i()); + (void)b; + return 0; + } + " HAVE_AVX512) + if (HAVE_AVX512 AND ENABLE_AVX512) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + + set (TEST_FLAG "-mbmi") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _blsr_u32(0); + (void)a; + return 0; + } + " HAVE_BMI) + if (HAVE_BMI AND ENABLE_BMI) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () endif () cmake_pop_check_state () diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 94127fa8eb3..9daec1c1c64 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -230,8 +230,74 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + col_size; const UInt8 * data_pos = chars.data(); +#if defined(__AVX512F__) && defined(__AVX512BW__) + static constexpr size_t SIMD_BYTES = 64; + const __m512i zero64 = _mm512_setzero_epi32(); + const UInt8 * filt_end_avx512 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; + const size_t chars_per_simd_elements = SIMD_BYTES * n; -#ifdef __SSE2__ + while (filt_pos < filt_end_avx512) + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + + if (0xFFFFFFFFFFFFFFFF == mask) + { + res->chars.insert(data_pos, data_pos + chars_per_simd_elements); + } + else + { + size_t res_chars_size = res->chars.size(); + while (mask) + { + size_t index = __builtin_ctzll(mask); + res->chars.resize(res_chars_size + n); + memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); + res_chars_size += n; + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + data_pos += chars_per_simd_elements; + filt_pos += SIMD_BYTES; + } +#elif defined(__AVX2__) + static constexpr size_t SIMD_BYTES = 32; + const __m256i zero32 = _mm256_setzero_si256(); + const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; + const size_t chars_per_simd_elements = SIMD_BYTES * n; + + while (filt_pos < filt_end_avx2) + { + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + + if (0xFFFFFFFF == mask) + { + res->chars.insert(data_pos, data_pos + chars_per_simd_elements); + } + else + { + size_t res_chars_size = res->chars.size(); + while (mask) + { + size_t index = __builtin_ctz(mask); + res->chars.resize(res_chars_size + n); + memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); + res_chars_size += n; + #ifdef __BMI__ + mask = _blsr_u32(mask); + #else + mask = mask & (mask-1); + #endif + } + } + data_pos += chars_per_simd_elements; + filt_pos += SIMD_BYTES; + } + +#elif defined(__SSE2__) /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -267,6 +333,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result data_pos += chars_per_simd_elements; filt_pos += SIMD_BYTES; } + #endif size_t res_chars_size = res->chars.size(); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 3ee692a3ff4..000a7198446 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -311,7 +311,67 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); -#ifdef __SSE2__ +#if defined(__AVX512F__) && defined(__AVX512BW__) + static constexpr size_t SIMD_BYTES = 64; + const __m512i zero64 = _mm512_setzero_epi32(); + const UInt8 * filt_end_avx512 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_avx512) + { + UInt64 mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + + if (0xFFFFFFFFFFFFFFFF == mask) + { + res_data.insert(data_pos, data_pos + SIMD_BYTES); + } + else + { + while (mask) + { + size_t index = __builtin_ctzll(mask); + res_data.push_back(data_pos[index]); + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + data_pos += SIMD_BYTES; + } +#elif defined(__AVX2__) + static constexpr size_t SIMD_BYTES = 32; + const __m256i zero32 = _mm256_setzero_si256(); + const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_avx2) + { + UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + + if (0xFFFFFFFF == mask) + { + res_data.insert(data_pos, data_pos + SIMD_BYTES); + } + else + { + while (mask) + { + size_t index = __builtin_ctz(mask); + res_data.push_back(data_pos[index]); + #ifdef __BMI__ + mask = _blsr_u32(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + data_pos += SIMD_BYTES; + } +#elif defined(__SSE2__) /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -344,6 +404,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } + #endif while (filt_pos < filt_end) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index a4d7de34382..36c292b4196 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -229,7 +229,89 @@ namespace memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T)); }; - #ifdef __SSE2__ + #if defined(__AVX512F__) && defined(__AVX512BW__) + const __m512i zero_vec = _mm512_setzero_epi32(); + static constexpr size_t SIMD_BYTES = 64; + const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_aligned) + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); + + if (mask == 0xffffffffffffffff) + { + /// SIMD_BYTES consecutive rows pass the filter + const auto first = offsets_pos == offsets_begin; + + const auto chunk_offset = first ? 0 : offsets_pos[-1]; + const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset; + + result_offsets_builder.template insertChunk(offsets_pos, first, chunk_offset, chunk_size); + + /// copy elements for SIMD_BYTES arrays at once + const auto elems_size_old = res_elems.size(); + res_elems.resize(elems_size_old + chunk_size); + memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T)); + } + else + { + while (mask) + { + size_t index = __builtin_ctzll(mask); + copy_array(offsets_pos + index); + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + offsets_pos += SIMD_BYTES; + } + #elif defined(__AVX2__) + const __m256i zero_vec = _mm256_setzero_si256(); + static constexpr size_t SIMD_BYTES = 32; + const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_aligned) + { + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); + + if (mask == 0xffffffff) + { + /// SIMD_BYTES consecutive rows pass the filter + const auto first = offsets_pos == offsets_begin; + + const auto chunk_offset = first ? 0 : offsets_pos[-1]; + const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset; + + result_offsets_builder.template insertChunk(offsets_pos, first, chunk_offset, chunk_size); + + /// copy elements for SIMD_BYTES arrays at once + const auto elems_size_old = res_elems.size(); + res_elems.resize(elems_size_old + chunk_size); + memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T)); + } + else + { + while (mask) + { + size_t index = __builtin_ctz(mask); + copy_array(offsets_pos + index); + #ifdef __BMI__ + mask = _blsr_u32(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + offsets_pos += SIMD_BYTES; + } + #elif defined(__SSE2__) const __m128i zero_vec = _mm_setzero_si128(); static constexpr size_t SIMD_BYTES = 16; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; @@ -268,7 +350,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #endif while (filt_pos < filt_end) From 2c92caac170ad4b2d0728fe6b3c33735da1457ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Oct 2021 21:20:35 +0300 Subject: [PATCH 455/950] Fix clickhouse local. --- src/Client/ClientBase.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 99a80901cdd..81e9cc47134 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -48,6 +48,7 @@ #include #include #include +#include namespace fs = std::filesystem; @@ -240,7 +241,7 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (need_render_progress && (stdout_is_a_tty || is_interactive)) progress_indication.clearProgressOutput(); - output_format->write(block); + output_format->write(materializeBlock(block)); written_first_block = true; /// Received data block is immediately displayed to the user. @@ -358,10 +359,10 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) current_format = "Vertical"; /// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly. - if (!need_render_progress) - output_format = global_context->getOutputFormatParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); - else - output_format = global_context->getOutputFormat(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + // if (!need_render_progress) + // output_format = global_context->getOutputFormatParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + // else + output_format = global_context->getOutputFormat(current_format, out_file_buf ? *out_file_buf : *out_buf, block); output_format->doWritePrefix(); } From 706e2b6b8816a3e9a5022341331227a1986987da Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 11 Oct 2021 21:42:46 +0300 Subject: [PATCH 456/950] more strict check for intersecting parts --- src/Storages/MergeTree/MergeTreePartInfo.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index df3d9cb9237..82fe0b860c8 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -60,11 +60,16 @@ struct MergeTreePartInfo /// True if contains rhs (this part is obtained by merging rhs with some other parts or mutating rhs) bool contains(const MergeTreePartInfo & rhs) const { + /// Containing part may have equal level iff block numbers are equal (unless level is MAX_LEVEL) + /// (e.g. all_0_5_2 does not contain all_0_4_2, but all_0_5_3 or all_0_4_2_9 do) + bool strictly_contains_block_range = (min_block == rhs.min_block && max_block == rhs.max_block) || level > rhs.level + || level == MAX_LEVEL || level == LEGACY_MAX_LEVEL; return partition_id == rhs.partition_id /// Parts for different partitions are not merged && min_block <= rhs.min_block && max_block >= rhs.max_block && level >= rhs.level - && mutation >= rhs.mutation; + && mutation >= rhs.mutation + && strictly_contains_block_range; } /// Return part mutation version, if part wasn't mutated return zero From f1791ddc44e9dfe6644aa1f406e6b3924374e725 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 11 Oct 2021 15:49:30 -0300 Subject: [PATCH 457/950] explanation for volume/max_data_part_size_bytes --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index f7118f7557e..4f473279067 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -688,7 +688,7 @@ Tags: - `policy_name_N` — Policy name. Policy names must be unique. - `volume_name_N` — Volume name. Volume names must be unique. - `disk` — a disk within a volume. -- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volume’s disks. +- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volume’s disks. If the a size of a merged part estimated to be bigger than `max_data_part_size_bytes` then this part will be written to a next volume. Basically this feature allows to keep new/small parts on a hot (SSD) volume and move them to a cold (HDD) volume when they reach large size. Do not use this setting if your policy has only one volume. - `move_factor` — when the amount of available space gets lower than this factor, data automatically start to move on the next volume if any (by default, 0.1). - `prefer_not_to_merge` — Disables merging of data parts on this volume. When this setting is enabled, merging data on this volume is not allowed. This allows controlling how ClickHouse works with slow disks. From fb83d2ddd593f914a465274566eb9bdca429ed55 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 11 Oct 2021 15:56:02 -0300 Subject: [PATCH 458/950] Update mergetree.md --- docs/ru/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index e8152441101..bef14924d36 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -668,7 +668,7 @@ TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); - `policy_name_N` — название политики. Названия политик должны быть уникальны. - `volume_name_N` — название тома. Названия томов должны быть уникальны. - `disk` — диск, находящийся внутри тома. -- `max_data_part_size_bytes` — максимальный размер куска данных, который может находится на любом из дисков этого тома. +- `max_data_part_size_bytes` — максимальный размер куска данных, который может находится на любом из дисков этого тома. Если в результате слияния размер куска ожидается больше, чем max_data_part_size_bytes, то этот кусок будет записан в следующий том. В основном эта функция позволяет хранить новые / мелкие куски на горячем (SSD) томе и перемещать их на холодный (HDD) том, когда они достигают большого размера. Не используйте этот параметр, если политика имеет только один том. - `move_factor` — доля доступного свободного места на томе, если места становится меньше, то данные начнут перемещение на следующий том, если он есть (по умолчанию 0.1). - `prefer_not_to_merge` — Отключает слияние кусков данных, хранящихся на данном томе. Если данная настройка включена, то слияние данных, хранящихся на данном томе, не допускается. Это позволяет контролировать работу ClickHouse с медленными дисками. From 1e1d5d7fea677f4a1b9f12eba5a1f0b1c19d8c58 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Oct 2021 22:21:04 +0300 Subject: [PATCH 459/950] Fix style. --- src/Formats/NullFormat.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/NullFormat.h b/src/Formats/NullFormat.h index 9822c9f5d16..7aa9102790f 100644 --- a/src/Formats/NullFormat.h +++ b/src/Formats/NullFormat.h @@ -1,3 +1,4 @@ +#pragma once #include namespace DB From 381d666e4b368ba3d750bafea5399a076a27a5f5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 11 Oct 2021 23:09:31 +0300 Subject: [PATCH 460/950] Ignore parallel removing warning in 00992_system_parts_race_condition_zookeeper_long CI: https://clickhouse-test-reports.s3.yandex.net/0/6c7fbf0b888db9c1272478189f0ff40212a3e7c9/functional_stateless_tests_(release,_databaseordinary)/test_run.txt.out.log --- .../00992_system_parts_race_condition_zookeeper_long.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 273b39961af..aee8a7727e5 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -79,7 +79,7 @@ timeout $TIMEOUT bash -c thread5 2> /dev/null & wait check_replication_consistency "alter_table" "count(), sum(a), sum(b), round(sum(c))" -$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" & -$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" & +$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & +$CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & wait From 83a9a8d4dc8239035c4a0ff9c85658d17c78131b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 11 Oct 2021 23:23:56 +0300 Subject: [PATCH 461/950] Update KeeperStateMachine.cpp --- src/Coordination/KeeperStateMachine.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index be4f73cf5ae..682a523fcaf 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -125,10 +125,6 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n } else { - LOG_TEST(log, "Commit request for session {} with type {}, log id {}{}", - request_for_session.session_id, toString(request_for_session.request->getOpNum()), log_idx, - request_for_session.request->getPath().empty() ? "" : ", path " + request_for_session.request->getPath()); - std::lock_guard lock(storage_and_responses_lock); KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session.request, request_for_session.session_id, log_idx); for (auto & response_for_session : responses_for_sessions) From 0ef26244083333ef4ee8ede0049014334a573459 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 04:04:27 +0530 Subject: [PATCH 462/950] add specific flags for compiling filter operation source files --- cmake/cpu_features.cmake | 4 ++-- src/CMakeLists.txt | 7 +++++++ src/Columns/ColumnFixedString.cpp | 11 ++++++----- src/Columns/ColumnVector.cpp | 15 ++++++++------- 4 files changed, 23 insertions(+), 14 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index d77ca0b32e3..4ea9465be98 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -143,7 +143,7 @@ else () } " HAVE_AVX512) if (HAVE_AVX512 AND ENABLE_AVX512) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") endif () set (TEST_FLAG "-mbmi") @@ -157,7 +157,7 @@ else () } " HAVE_BMI) if (HAVE_BMI AND ENABLE_BMI) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") endif () endif () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..45bb1a21d59 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,6 +284,13 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) +# Use X86 AVX2/AVX512 instructions to accelerate filter opertions +set_source_files_properties( + Columns/ColumnFixedString.cpp + Columns/ColumnsCommon.cpp + Columns/ColumnVector.cpp + PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") + if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) endif() diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 9daec1c1c64..4b31677d37e 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -230,6 +230,12 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + col_size; const UInt8 * data_pos = chars.data(); + + /** A slightly more optimized version. + * Based on the assumption that often pieces of consecutive values + * completely pass or do not pass the filter. + * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; const __m512i zero64 = _mm512_setzero_epi32(); @@ -298,11 +304,6 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result } #elif defined(__SSE2__) - /** A slightly more optimized version. - * Based on the assumption that often pieces of consecutive values - * completely pass or do not pass the filter. - * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. - */ static constexpr size_t SIMD_BYTES = 16; const __m128i zero16 = _mm_setzero_si128(); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 000a7198446..37f60e9f2b9 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -310,7 +310,12 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); - + + /** A slightly more optimized version. + * Based on the assumption that often pieces of consecutive values + * completely pass or do not pass the filter. + * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; const __m512i zero64 = _mm512_setzero_epi32(); @@ -341,6 +346,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } + #elif defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; const __m256i zero32 = _mm256_setzero_si256(); @@ -371,13 +377,8 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } -#elif defined(__SSE2__) - /** A slightly more optimized version. - * Based on the assumption that often pieces of consecutive values - * completely pass or do not pass the filter. - * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. - */ +#elif defined(__SSE2__) static constexpr size_t SIMD_BYTES = 16; const __m128i zero16 = _mm_setzero_si128(); const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From d4e496c31424bfe428626ac3a18e5de1b13e13dc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 01:47:34 +0300 Subject: [PATCH 463/950] Add support for PowerPC build --- cmake/linux/default_libs.cmake | 2 +- cmake/target.cmake | 13 +- contrib/CMakeLists.txt | 2 +- .../internal/jemalloc_internal_defs.h.in | 4 +- contrib/libuv-cmake/CMakeLists.txt | 160 ++++++++++++++++++ src/Interpreters/ITokenExtractor.cpp | 2 +- utils/CMakeLists.txt | 2 +- utils/memcpy-bench/CMakeLists.txt | 2 +- 8 files changed, 175 insertions(+), 12 deletions(-) create mode 100644 contrib/libuv-cmake/CMakeLists.txt diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index c1e4d450389..a2da7ba1915 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -5,7 +5,7 @@ set (DEFAULT_LIBS "-nodefaultlibs") # We need builtins from Clang's RT even without libcxx - for ubsan+int128. # See https://bugs.llvm.org/show_bug.cgi?id=16404 -if (COMPILER_CLANG AND NOT (CMAKE_CROSSCOMPILING AND ARCH_AARCH64)) +if (COMPILER_CLANG AND NOT CMAKE_CROSSCOMPILING) execute_process (COMMAND ${CMAKE_CXX_COMPILER} --print-libgcc-file-name --rtlib=compiler-rt OUTPUT_VARIABLE BUILTINS_LIBRARY OUTPUT_STRIP_TRAILING_WHITESPACE) else () set (BUILTINS_LIBRARY "-lgcc") diff --git a/cmake/target.cmake b/cmake/target.cmake index d1a0b8f9cbf..ca6009e68d3 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -34,11 +34,14 @@ if (CMAKE_CROSSCOMPILING) # FIXME: broken dependencies set (ENABLE_PROTOBUF OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") - - set (ENABLE_PARQUET OFF CACHE INTERNAL "") - set (ENABLE_ORC OFF CACHE INTERNAL "") - - set (ENABLE_MYSQL OFF CACHE INTERNAL "") + set (USE_SENTRY OFF CACHE INTERNAL "") +# set (ENABLE_ROCKSDB OFF CACHE INTERNAL "") + endif () + elseif (ARCH_PPC64LE) + set (ENABLE_PROTOBUF OFF CACHE INTERNAL "") + set (ENABLE_GRPC OFF CACHE INTERNAL "") + set (USE_SENTRY OFF CACHE INTERNAL "") +# set (ENABLE_ROCKSDB OFF CACHE INTERNAL "") endif () elseif (OS_FREEBSD) # FIXME: broken dependencies diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 4d817c4c6e2..5ff85fa85c2 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -278,7 +278,7 @@ if (USE_FASTOPS) endif() if (USE_AMQPCPP OR USE_CASSANDRA) - add_subdirectory (libuv) + add_subdirectory (libuv-cmake) endif() if (USE_AMQPCPP) add_subdirectory (amqpcpp-cmake) diff --git a/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in index 8068861041f..97d0d4d8471 100644 --- a/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in @@ -81,7 +81,7 @@ /* #undef JEMALLOC_HAVE_ISSETUGID */ /* Defined if pthread_atfork(3) is available. */ -#define JEMALLOC_HAVE_PTHREAD_ATFORK +/* #undef JEMALLOC_HAVE_PTHREAD_ATFORK */ /* Defined if pthread_setname_np(3) is available. */ #define JEMALLOC_HAVE_PTHREAD_SETNAME_NP @@ -284,7 +284,7 @@ #define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS /* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ -/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ +#define JEMALLOC_DEFINE_MADVISE_FREE /* * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt new file mode 100644 index 00000000000..4fbd0575b55 --- /dev/null +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -0,0 +1,160 @@ +# This file is a modified version of contrib/libuv/CMakeLists.txt + +include(CMakeDependentOption) + +set (SOURCE_DIR "${CMAKE_SOURCE_DIR}/contrib/libuv") +set (BINARY_DIR "${CMAKE_BINARY_DIR}/contrib/libuv") + + +if(CMAKE_C_COMPILER_ID MATCHES "AppleClang|Clang|GNU") + list(APPEND uv_cflags -fvisibility=hidden --std=gnu89) + list(APPEND uv_cflags -Wall -Wextra -Wstrict-prototypes) + list(APPEND uv_cflags -Wno-unused-parameter) +endif() + +set(uv_sources + src/fs-poll.c + src/idna.c + src/inet.c + src/random.c + src/strscpy.c + src/threadpool.c + src/timer.c + src/uv-common.c + src/uv-data-getter-setters.c + src/version.c + src/unix/async.c + src/unix/core.c + src/unix/dl.c + src/unix/fs.c + src/unix/getaddrinfo.c + src/unix/getnameinfo.c + src/unix/loop-watcher.c + src/unix/loop.c + src/unix/pipe.c + src/unix/poll.c + src/unix/process.c + src/unix/random-devurandom.c + src/unix/signal.c + src/unix/stream.c + src/unix/tcp.c + src/unix/thread.c + src/unix/tty.c + src/unix/udp.c) + +if(APPLE OR CMAKE_SYSTEM_NAME MATCHES "Android|Linux|OS/390") + list(APPEND uv_sources src/unix/proctitle.c) +endif() + +if(CMAKE_SYSTEM_NAME MATCHES "DragonFly|FreeBSD") + list(APPEND uv_sources src/unix/freebsd.c) +endif() + +if(CMAKE_SYSTEM_NAME MATCHES "DragonFly|FreeBSD|NetBSD|OpenBSD") + list(APPEND uv_sources src/unix/posix-hrtime.c src/unix/bsd-proctitle.c) +endif() + +if(APPLE OR CMAKE_SYSTEM_NAME MATCHES "DragonFly|FreeBSD|NetBSD|OpenBSD") + list(APPEND uv_sources src/unix/bsd-ifaddrs.c src/unix/kqueue.c) +endif() + +if(CMAKE_SYSTEM_NAME MATCHES "FreeBSD") + list(APPEND uv_sources src/unix/random-getrandom.c) +endif() + +if(APPLE OR CMAKE_SYSTEM_NAME STREQUAL "OpenBSD") + list(APPEND uv_sources src/unix/random-getentropy.c) +endif() + +if(APPLE) + list(APPEND uv_defines _DARWIN_UNLIMITED_SELECT=1 _DARWIN_USE_64_BIT_INODE=1) + list(APPEND uv_sources + src/unix/darwin-proctitle.c + src/unix/darwin.c + src/unix/fsevents.c) +endif() + +if(CMAKE_SYSTEM_NAME STREQUAL "Linux") + list(APPEND uv_defines _GNU_SOURCE _POSIX_C_SOURCE=200112) + list(APPEND uv_libraries dl rt) + list(APPEND uv_sources + src/unix/linux-core.c + src/unix/linux-inotify.c + src/unix/linux-syscalls.c + src/unix/procfs-exepath.c + src/unix/random-getrandom.c + src/unix/random-sysctl-linux.c + src/unix/sysinfo-loadavg.c) +endif() + +if(CMAKE_SYSTEM_NAME STREQUAL "NetBSD") + list(APPEND uv_sources src/unix/netbsd.c) + list(APPEND uv_libraries kvm) +endif() + +if(CMAKE_SYSTEM_NAME STREQUAL "OpenBSD") + list(APPEND uv_sources src/unix/openbsd.c) +endif() + +if(CMAKE_SYSTEM_NAME STREQUAL "OS/390") + list(APPEND uv_defines PATH_MAX=255) + list(APPEND uv_defines _AE_BIMODAL) + list(APPEND uv_defines _ALL_SOURCE) + list(APPEND uv_defines _LARGE_TIME_API) + list(APPEND uv_defines _OPEN_MSGQ_EXT) + list(APPEND uv_defines _OPEN_SYS_FILE_EXT) + list(APPEND uv_defines _OPEN_SYS_IF_EXT) + list(APPEND uv_defines _OPEN_SYS_SOCK_EXT3) + list(APPEND uv_defines _OPEN_SYS_SOCK_IPV6) + list(APPEND uv_defines _UNIX03_SOURCE) + list(APPEND uv_defines _UNIX03_THREADS) + list(APPEND uv_defines _UNIX03_WITHDRAWN) + list(APPEND uv_defines _XOPEN_SOURCE_EXTENDED) + list(APPEND uv_sources + src/unix/pthread-fixes.c + src/unix/pthread-barrier.c + src/unix/os390.c + src/unix/os390-syscalls.c) +endif() + +if(CMAKE_SYSTEM_NAME STREQUAL "SunOS") + list(APPEND uv_defines __EXTENSIONS__ _XOPEN_SOURCE=500) + list(APPEND uv_libraries kstat nsl sendfile socket) + list(APPEND uv_sources src/unix/no-proctitle.c src/unix/sunos.c) +endif() + +set(uv_sources_tmp "") +foreach(file ${uv_sources}) + list(APPEND uv_sources_tmp "${SOURCE_DIR}/${file}") +endforeach(file) +set(uv_sources "${uv_sources_tmp}") + +list(APPEND uv_defines CLICKHOUSE_GLIBC_COMPATIBILITY) + +add_library(uv ${uv_sources}) +target_compile_definitions(uv + INTERFACE USING_UV_SHARED=1 + PRIVATE ${uv_defines} BUILDING_UV_SHARED=1) +target_compile_options(uv PRIVATE ${uv_cflags}) +target_include_directories(uv PUBLIC ${SOURCE_DIR}/include PRIVATE ${SOURCE_DIR}/src) +target_link_libraries(uv ${uv_libraries}) + +add_library(uv_a STATIC ${uv_sources}) +target_compile_definitions(uv_a PRIVATE ${uv_defines}) +target_compile_options(uv_a PRIVATE ${uv_cflags}) +target_include_directories(uv_a PUBLIC ${SOURCE_DIR}/include PRIVATE ${SOURCE_DIR}/src) +target_link_libraries(uv_a ${uv_libraries}) + +if(UNIX) + # Now for some gibbering horrors from beyond the stars... + foreach(x ${uv_libraries}) + set(LIBS "${LIBS} -l${x}") + endforeach(x) + file(STRINGS ${SOURCE_DIR}/configure.ac configure_ac REGEX ^AC_INIT) + string(REGEX MATCH [0-9]+[.][0-9]+[.][0-9]+ PACKAGE_VERSION "${configure_ac}") + string(REGEX MATCH ^[0-9]+ UV_VERSION_MAJOR "${PACKAGE_VERSION}") + # The version in the filename is mirroring the behaviour of autotools. + set_target_properties(uv PROPERTIES VERSION ${UV_VERSION_MAJOR}.0.0 + SOVERSION ${UV_VERSION_MAJOR}) +endif() + diff --git a/src/Interpreters/ITokenExtractor.cpp b/src/Interpreters/ITokenExtractor.cpp index 83166079e89..8c1af130f71 100644 --- a/src/Interpreters/ITokenExtractor.cpp +++ b/src/Interpreters/ITokenExtractor.cpp @@ -6,7 +6,7 @@ #include #if defined(__SSE2__) -#include +#include #if defined(__SSE4_2__) #include diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index a6bf2843e9a..8309b6bcb53 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -38,7 +38,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) endif () # memcpy_jart.S contains position dependent code - if (NOT CMAKE_POSITION_INDEPENDENT_CODE AND NOT OS_DARWIN AND NOT OS_SUNOS AND NOT ARCH_AARCH64) + if (NOT CMAKE_POSITION_INDEPENDENT_CODE AND OS_LINUX AND ARCH_AMD64) add_subdirectory (memcpy-bench) endif () endif () diff --git a/utils/memcpy-bench/CMakeLists.txt b/utils/memcpy-bench/CMakeLists.txt index 5353b6fb68e..593a359a876 100644 --- a/utils/memcpy-bench/CMakeLists.txt +++ b/utils/memcpy-bench/CMakeLists.txt @@ -16,7 +16,7 @@ add_executable (memcpy-bench add_compile_options(memcpy-bench PRIVATE -fno-tree-loop-distribute-patterns) if (OS_SUNOS) - target_compile_options(memcpy-bench PRIVATE "-Wa,--divide") + target_compile_options(memcpy-bench PRIVATE "-Wa,--divide") endif() set_source_files_properties(FastMemcpy.cpp PROPERTIES COMPILE_FLAGS "-Wno-old-style-cast") From 0c6c716bdd7c18bb2fe7cabcadaa0189eb8e7457 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 01:49:29 +0300 Subject: [PATCH 464/950] Update submodule --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index 611d3315e9e..002415524b5 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 611d3315e9e369a338de4ffa128eb87b4fb87dec +Subproject commit 002415524b5d14124bb8a61a3ce7ac65774f5479 From d809fa045605fc858ddaf80c50055fdc75ed87ed Mon Sep 17 00:00:00 2001 From: Ivan Milov Date: Tue, 12 Oct 2021 01:52:42 +0300 Subject: [PATCH 465/950] Edited and translated to Russian Edit the English version of changes and translate to Russian where needed. --- .../settings.md | 22 +-- .../functions/type-conversion-functions.md | 10 +- docs/ru/interfaces/formats.md | 2 +- .../settings.md | 6 +- .../functions/date-time-functions.md | 2 +- .../functions/type-conversion-functions.md | 165 +++++++++++++++++- 6 files changed, 184 insertions(+), 23 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 425853553ed..7a157dbe946 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -71,13 +71,13 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression. ## encryption {#server-settings-encryption} -Configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Key (or keys) should be written in enviroment variables or be set in configuration file. +It configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Key (or keys) should be written in environment variables or set in the configuration file. -Keys can be hex or string. Their length must be equal to 16. +Keys can be hex or string with the length equal to 16 bytes. **Example** -Load from config: +Loading from config: ```xml @@ -88,9 +88,9 @@ Load from config: ``` !!! note "NOTE" - Storing keys in configuration file is not recommended. It isn't secure. You can move the keys into a separate config file on a secure disk and put a symlink to that config file to `config.d/` folder. + Storing keys in the configuration file is not recommended. It isn't secure. You can move the keys into a separate config file on a secure disk and put a symlink to that config file to `config.d/` folder. -Load from config, when key is in hex: +Loading from config, when the key is in hex: ```xml @@ -100,7 +100,7 @@ Load from config, when key is in hex: ``` -Load key from environment variable: +Loading key from the environment variable: ```xml @@ -110,9 +110,9 @@ Load key from environment variable: ``` -Where `current_key_id` sets the current key for encryption, and all specified keys can be used for decryption. +Here `current_key_id` sets the current key for encryption, and all specified keys can be used for decryption. -All this methods can be applied for multiple keys: +Each of these methods can be applied for multiple keys: ```xml @@ -124,9 +124,9 @@ All this methods can be applied for multiple keys: ``` -Where `current_key_id` shows current key for encryption. +Here `current_key_id` shows current key for encryption. -Also user can add nonce that must be 12 bytes long (by default encryption and decryption will use nonce consisting of zero bytes): +Also, users can add nonce that must be 12 bytes long (by default encryption and decryption processes use nonce that consists of zero bytes): ```xml @@ -146,7 +146,7 @@ Or it can be set in hex: ``` -Everything above can be applied for `aes_256_gcm_siv` (but key must be 32 bytes length). +Everything mentioned above can be applied for `aes_256_gcm_siv` (but the key must be 32 bytes long). ## custom_settings_prefixes {#custom_settings_prefixes} diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 19fa41c56cb..6385a1e650e 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -279,7 +279,7 @@ Result: └──────────────────────────────┴────────────────────┘ ``` -## toDate32OrDefault {#todate32-or-null} +## toDate32OrDefault {#todate32-or-default} The same as [toDate32](#todate32) but returns default value if invalid argument is received. @@ -362,7 +362,7 @@ Result: ``` -## toDecimal(32\|64\|128\|256)OrDefault {#todecimal3264128256ornull} +## toDecimal(32\|64\|128\|256)OrDefault {#todecimal3264128256ordefault} Converts an input string to a [Decimal(P,S)](../../sql-reference/data-types/decimal.md) data type value. This family of functions include: @@ -855,14 +855,14 @@ Result: ``` -## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_null} +## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_default} Converts input value `x` to the specified data type `T`. Returns default type value or `default_value` if specified if the casted value is not representable in the target type. **Syntax** ```sql -accurateCastOrNull(x, T) +accurateCastOrDefault(x, T) ``` **Parameters** @@ -873,7 +873,7 @@ accurateCastOrNull(x, T) **Returned value** -- The value, converted to the specified data type `T`. +- The value converted to the specified data type `T`. **Example** diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index d4cfa8a053c..712cc425c5b 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -73,7 +73,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT Формат `TabSeparated` поддерживает вывод тотальных значений (при использовании WITH TOTALS) и экстремальных значений (при настройке extremes выставленной в 1). В этих случаях, после основных данных выводятся тотальные значения, и экстремальные значения. Основной результат, тотальные значения и экстремальные значения, отделяются друг от друга пустой строкой. Пример: ``` sql -SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated`` +SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT TabSeparated ``` ``` text diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index bab67b62752..35062d5cb48 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -71,9 +71,9 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ## encryption {#server-settings-encryption} -Настраивает комманду для получения ключа используемого [кодеками шифрования](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Ключ (или несколько ключей) должны быть записаны в переменные окружения или установлены в конфигурационном файле . +Настраивает команду для получения ключа, используемого [кодеками шифрования](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Ключ (или несколько ключей) должен быть записан в переменные окружения или установлен в конфигурационном файле. -Ключи могут быть представлены в шестнадцатеричной или строковой форме. Их длинна должна быть равна 16. +Ключи могут быть представлены в шестнадцатеричной или строковой форме. Их длина должна быть равна 16 байтам. **Пример** @@ -146,7 +146,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part ``` -Все вышеперечисленное можно применить также для алгоритма `aes_256_gcm_siv` (но ключ должен быть длинной 32 байта). +Всё вышеперечисленное также применимо для алгоритма `aes_256_gcm_siv` (но ключ должен быть длиной 32 байта). ## custom_settings_prefixes {#custom_settings_prefixes} diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 282962b9e3f..599d43078b4 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -334,7 +334,7 @@ SELECT toStartOfISOYear(toDate('2017-01-01')) AS ISOYear20170101; **Синтаксис** ``` sql -toStartOfSecond(value[, timezone]) +toStartOfSecond(value, [timezone]) ``` **Аргументы** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 32f24d1e6c5..772f07fe8ec 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -90,6 +90,27 @@ SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); └─────────────────────────┴───────────────────────────┘ ``` +## toInt(8\|16\|32\|64\|128\|256)OrDefault {#toint8163264128256orDefault} + +Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64\|128\|256). Если не удалось - возвращает значение по умолчанию. + +**Пример** + +Запрос: + +``` sql +SELECT toInt64OrDefault('123123', cast('-1' as Int64)), toInt8OrDefault('123qwe123', cast('-1' as Int8)); +``` + +Результат: + +``` text +┌─toInt64OrDefault('123123', CAST('-1', 'Int64'))─┬─toInt8OrDefault('123qwe123', CAST('-1', 'Int8'))─┐ +│ 123123 │ -1 │ +└─────────────────────────────────────────────────┴──────────────────────────────────────────────────┘ +``` + + ## toUInt(8\|16\|32\|64\|256) {#touint8163264} Преобраует входное значение к типу [UInt](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает: @@ -132,12 +153,16 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); ## toUInt(8\|16\|32\|64\|256)OrNull {#touint8163264ornull} +## toUInt(8\|16\|32\|64\|256)OrDefault {#touint8163264256ordefault} + ## toFloat(32\|64) {#tofloat3264} ## toFloat(32\|64)OrZero {#tofloat3264orzero} ## toFloat(32\|64)OrNull {#tofloat3264ornull} +## toFloat(32\|64)OrDefault {#tofloat3264ordefault} + ## toDate {#todate} Cиноним: `DATE`. @@ -146,23 +171,27 @@ Cиноним: `DATE`. ## toDateOrNull {#todateornull} +## toDateOrDefault {#todateordefault} + ## toDateTime {#todatetime} ## toDateTimeOrZero {#todatetimeorzero} ## toDateTimeOrNull {#todatetimeornull} +## toDateTimeOrDefault {#todatetimeordefault} + ## toDate32 {#todate32} Конвертирует аргумент в значение типа [Date32](../../sql-reference/data-types/date32.md). Если значение выходит за границы диапазона, возвращается пограничное значение `Date32`. Если аргумент имеет тип [Date](../../sql-reference/data-types/date.md), учитываются границы типа `Date`. -**Синтаксис** +**Синтаксис** ``` sql toDate32(value) ``` -**Аргументы** +**Аргументы** - `value` — Значение даты. [String](../../sql-reference/data-types/string.md), [UInt32](../../sql-reference/data-types/int-uint.md) или [Date](../../sql-reference/data-types/date.md). @@ -250,6 +279,26 @@ SELECT toDate32OrNull('1955-01-01'), toDate32OrNull(''); └──────────────────────────────┴────────────────────┘ ``` +## toDate32OrDefault {#todate32-or-null} + +То же самое, что и [toDate32](#todate32), но возвращает значение по умолчанию, если получен недопустимый аргумент. + +**Пример** + +Запрос: + +``` sql +SELECT toDate32OrDefault('1955-01-01'), toDate32OrDefault(''); +``` + +Результат: + +``` text +┌─toDate32OrDefault('1955-01-01')─┬─toDate32OrDefault('')─┐ +│ 1955-01-01 │ 1970-01-01 │ +└─────────────────────────────────┴───────────────────────┘ +``` + ## toDecimal(32\|64\|128\|256) {#todecimal3264128} Преобразует `value` к типу данных [Decimal](../../sql-reference/functions/type-conversion-functions.md) с точностью `S`. `value` может быть числом или строкой. Параметр `S` (scale) задаёт число десятичных знаков. @@ -312,6 +361,60 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); └──────┴────────────────────────────────────────────────────┘ ``` + +## toDecimal(32\|64\|128\|256)OrDefault {#todecimal3264128256ornull} + +Преобразует входную строку в значение с типом данных [Decimal(P,S)](../../sql-reference/data-types/decimal.md). Семейство функций включает в себя: + +- `toDecimal32OrDefault(expr, S)` — Возвращает значение типа `Decimal32(S)` data type. +- `toDecimal64OrDefault(expr, S)` — Возвращает значение типа `Decimal64(S)` data type. +- `toDecimal128OrDefault(expr, S)` — Возвращает значение типа `Decimal128(S)` data type. +- `toDecimal256OrDefault(expr, S)` — Возвращает значение типа `Decimal256(S)` data type. + +Эти функции следует использовать вместо функций `toDecimal*()`, если при ошибке обработки входного значения вы хотите получать значение по умолчанию вместо исключения. + +**Аргументы** + +- `expr` — [выражение](../syntax.md#syntax-expressions), возвращающее значение типа [String](../../sql-reference/functions/type-conversion-functions.md). ClickHouse ожидает текстовое представление десятичного числа. Например, `'1.111'`. +- `S` — количество десятичных знаков в результирующем значении. + +**Возвращаемое значение** + +Значение типа `Nullable(Decimal(P,S))`. Значение содержит: + +- Число с `S` десятичными знаками, если ClickHouse распознал число во входной строке. +- `NULL`, если ClickHouse не смог распознать число во входной строке или входное число содержит больше чем `S` десятичных знаков. + +**Примеры** + +Запрос: + +``` sql +SELECT toDecimal32OrDefault(toString(-1.111), 5) AS val, toTypeName(val); +``` + +Результат: + +``` text +┌────val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 5))─┐ +│ -1.111 │ Decimal(9, 5) │ +└────────┴───────────────────────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT toDecimal32OrDefault(toString(-1.111), 2) AS val, toTypeName(val); +``` + +Результат: + +``` text +┌─val─┬─toTypeName(toDecimal32OrDefault(toString(-1.111), 2))─┐ +│ 0 │ Decimal(9, 2) │ +└─────┴───────────────────────────────────────────────────────┘ +``` + ## toDecimal(32\|64\|128\|256)OrZero {#todecimal3264128orzero} Преобразует тип входного значения в [Decimal (P, S)](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает в себя: @@ -750,6 +853,64 @@ SELECT └───────┴──────┴──────────────┘ ``` + + +## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_null} + +Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если указано. + +**Синтаксис** + +```sql +accurateCastOrDefault(x, T) +``` + +**Аргументы** + +- `x` — входное значение. +- `T` — имя возвращаемого типа данных. +- `default_value` - значение по умолчанию возвращаемого типа данных. + +**Возвращаемое значение** + +- Значение, преобразованное в указанный тип `T`. + +**Пример** + +Запрос: + +``` sql +SELECT toTypeName(accurateCastOrDefault(5, 'UInt8')); +``` + +Результат: + +``` text +┌─toTypeName(accurateCastOrDefault(5, 'UInt8'))─┐ +│ UInt8 │ +└───────────────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT + accurateCastOrDefault(-1, 'UInt8') as uint8, + accurateCastOrDefault(-1, 'UInt8', 5) as uint8_default, + accurateCastOrDefault(128, 'Int8') as int8, + accurateCastOrDefault(128, 'Int8', 5) as int8_default, + accurateCastOrDefault('Test', 'FixedString(2)') as fixed_string, + accurateCastOrDefault('Test', 'FixedString(2)', 'Te') as fixed_string_default; +``` + +Результат: + +``` text +┌─uint8─┬─uint8_default─┬─int8─┬─int8_default─┬─fixed_string─┬─fixed_string_default─┐ +│ 0 │ 5 │ 0 │ 5 │ │ Te │ +└───────┴───────────────┴──────┴──────────────┴──────────────┴──────────────────────┘ +``` + ## toInterval(Year\|Quarter\|Month\|Week\|Day\|Hour\|Minute\|Second) {#function-tointerval} Приводит аргумент из числового типа данных к типу данных [IntervalType](../../sql-reference/data-types/special-data-types/interval.md). From 54f3d0d2d977d62bee1a75510974326b9dc91047 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 02:02:44 +0300 Subject: [PATCH 466/950] Fix error --- cmake/target.cmake | 1 - 1 file changed, 1 deletion(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index ca6009e68d3..872202f2f29 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -36,7 +36,6 @@ if (CMAKE_CROSSCOMPILING) set (ENABLE_GRPC OFF CACHE INTERNAL "") set (USE_SENTRY OFF CACHE INTERNAL "") # set (ENABLE_ROCKSDB OFF CACHE INTERNAL "") - endif () elseif (ARCH_PPC64LE) set (ENABLE_PROTOBUF OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") From fa14dbdf429c06c3acc64311f0b49948028b6f51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 02:03:00 +0300 Subject: [PATCH 467/950] Update submodules --- contrib/boost | 2 +- contrib/libuv | 2 +- contrib/s2geometry | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/boost b/contrib/boost index 66d17f060c4..311cfd49896 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 66d17f060c4867aeea99fa2a20cfdae89ae2a2ec +Subproject commit 311cfd498966d4f77742703d605d9c2e7b4cc6a8 diff --git a/contrib/libuv b/contrib/libuv index e2e9b7e9f97..95081e7c16c 160000 --- a/contrib/libuv +++ b/contrib/libuv @@ -1 +1 @@ -Subproject commit e2e9b7e9f978ce8a1367b5fe781d97d1ce9f94ab +Subproject commit 95081e7c16c9857babe6d4e2bc1c779198ea89ae diff --git a/contrib/s2geometry b/contrib/s2geometry index 20ea540d81f..38b7a290f92 160000 --- a/contrib/s2geometry +++ b/contrib/s2geometry @@ -1 +1 @@ -Subproject commit 20ea540d81f4575a3fc0aea585aac611bcd03ede +Subproject commit 38b7a290f927cc372218c2094602b83e35b18c05 From 96de70c0005bcb88666c82f1f36e9a1c05bfa0fe Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Mon, 11 Oct 2021 17:06:07 -0600 Subject: [PATCH 468/950] Adjusted documentation to align stripe size with write size, and correct block size to common RAID recommendations --- docs/en/operations/tips.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 27fc7d47a76..5cbbe71b3e0 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -1,4 +1,3 @@ ---- toc_priority: 58 toc_title: Usage Recommendations --- @@ -60,7 +59,7 @@ $ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size Calculate the exact number from the number of devices and the block size, using the formula: `2 * num_devices * chunk_size_in_bytes / 4096`. -A block size of 1024 bytes (1KB) is sufficient for most RAID configurations. A block size range of 1KB to 16KB is common across many storage system vendors or DBMS. +A block size of 64 KB is sufficient for most RAID configurations. The average clickhouse-server write size is approximately 1 MB (1024 KB), and thus the recommended stripe size is also 1 MB. The block size can be optimized if needed when set to 1 MB divided by the number of non-parity disks in the RAID array, such that each write is parallelized across all available non-parity disks. Never set the block size too small or too large. You can use RAID-0 on SSD. From e1409c143b7d05a31bd26b8ebf063a160ce8b87d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 02:06:22 +0300 Subject: [PATCH 469/950] Add toolchain file --- cmake/linux/toolchain-ppc64le.cmake | 32 +++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 cmake/linux/toolchain-ppc64le.cmake diff --git a/cmake/linux/toolchain-ppc64le.cmake b/cmake/linux/toolchain-ppc64le.cmake new file mode 100644 index 00000000000..cf85fc20fc4 --- /dev/null +++ b/cmake/linux/toolchain-ppc64le.cmake @@ -0,0 +1,32 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + +set (CMAKE_SYSTEM_NAME "Linux") +set (CMAKE_SYSTEM_PROCESSOR "ppc64le") +set (CMAKE_C_COMPILER_TARGET "ppc64le-linux-gnu") +set (CMAKE_CXX_COMPILER_TARGET "ppc64le-linux-gnu") +set (CMAKE_ASM_COMPILER_TARGET "ppc64le-linux-gnu") + +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-powerpc64le") + +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/powerpc64le-linux-gnu/libc") + +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) From cb9bdf9666b8e1efac8b1d96e62359622c5bfd65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 02:07:03 +0300 Subject: [PATCH 470/950] Minor change --- cmake/target.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index 872202f2f29..e8932a893c0 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -55,7 +55,7 @@ if (CMAKE_CROSSCOMPILING) endif () # Don't know why but CXX_STANDARD doesn't work for cross-compilation - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++17") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=c++20") message (STATUS "Cross-compiling for target: ${CMAKE_CXX_COMPILE_TARGET}") endif () From 75547e64a5ce564bcac8fd1446af5155f6a25822 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 02:10:58 +0300 Subject: [PATCH 471/950] Add to packager --- docker/packager/packager | 9 +++++++-- tests/ci/ci_config.json | 10 ++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index f8e5fd717cf..ae7b99200ee 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -61,6 +61,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ DARWIN_ARM_SUFFIX = "-darwin-aarch64" ARM_SUFFIX = "-aarch64" FREEBSD_SUFFIX = "-freebsd" + PPC_SUFFIX = '-ppc64le' result = [] cmake_flags = ['$CMAKE_FLAGS', '-DADD_GDB_INDEX_FOR_GOLD=1'] @@ -69,8 +70,9 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ is_cross_darwin = compiler.endswith(DARWIN_SUFFIX) is_cross_darwin_arm = compiler.endswith(DARWIN_ARM_SUFFIX) is_cross_arm = compiler.endswith(ARM_SUFFIX) + is_cross_ppc = compiler.endswith(PPC_SUFFIX) is_cross_freebsd = compiler.endswith(FREEBSD_SUFFIX) - is_cross_compile = is_cross_darwin or is_cross_darwin_arm or is_cross_arm or is_cross_freebsd + is_cross_compile = is_cross_darwin or is_cross_darwin_arm or is_cross_arm or is_cross_freebsd or is_cross_ppc # Explicitly use LLD with Clang by default. # Don't force linker for cross-compilation. @@ -97,6 +99,9 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ elif is_cross_freebsd: cc = compiler[:-len(FREEBSD_SUFFIX)] cmake_flags.append("-DCMAKE_TOOLCHAIN_FILE=/build/cmake/freebsd/toolchain-x86_64.cmake") + elif is_cross_ppc: + cc = compiler[:-len(PPC_SUFFIX)] + cmake_flags.append("-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-ppc64le.cmake") else: cc = compiler @@ -205,7 +210,7 @@ if __name__ == "__main__": parser.add_argument("--build-type", choices=("debug", ""), default="") parser.add_argument("--compiler", choices=("clang-11", "clang-11-darwin", "clang-11-darwin-aarch64", "clang-11-aarch64", "clang-12", "clang-12-darwin", "clang-12-darwin-aarch64", "clang-12-aarch64", - "clang-13", "clang-13-darwin", "clang-13-darwin-aarch64", "clang-13-aarch64", + "clang-13", "clang-13-darwin", "clang-13-darwin-aarch64", "clang-13-aarch64", "clang-13-ppc64le", "clang-11-freebsd", "clang-12-freebsd", "clang-13-freebsd", "gcc-11"), default="clang-13") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") parser.add_argument("--unbundled", action="store_true") diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 6222e4f61bc..4feae56b93c 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -162,6 +162,16 @@ "splitted": "unsplitted", "tidy": "disable", "with_coverage": false + }, + { + "compiler": "clang-13-ppc64le", + "build-type": "", + "sanitizer": "", + "package-type": "binary", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": false } ], "tests_config": { From 0d076468666df08c950c7630d8b126d693bfc32e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 03:17:35 +0300 Subject: [PATCH 472/950] Fix strange code --- cmake/find/ssl.cmake | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/cmake/find/ssl.cmake b/cmake/find/ssl.cmake index fd6ed56dcdb..fdc0bfb27d3 100644 --- a/cmake/find/ssl.cmake +++ b/cmake/find/ssl.cmake @@ -53,12 +53,7 @@ endif () if (NOT OPENSSL_FOUND AND NOT MISSING_INTERNAL_SSL_LIBRARY) set (USE_INTERNAL_SSL_LIBRARY 1) set (OPENSSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/boringssl") - - if (ARCH_AMD64) - set (OPENSSL_INCLUDE_DIR "${OPENSSL_ROOT_DIR}/include") - elseif (ARCH_AARCH64) - set (OPENSSL_INCLUDE_DIR "${OPENSSL_ROOT_DIR}/include") - endif () + set (OPENSSL_INCLUDE_DIR "${OPENSSL_ROOT_DIR}/include") set (OPENSSL_CRYPTO_LIBRARY crypto) set (OPENSSL_SSL_LIBRARY ssl) set (OPENSSL_FOUND 1) From 7b9358a59721749b7a60d917d1793969760a9df5 Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Tue, 12 Oct 2021 03:21:31 +0300 Subject: [PATCH 473/950] adding codegen fuzzer + code generation script. new contrib added: libprotobuf-mutator --- .gitmodules | 3 + CMakeLists.txt | 1 + cmake/find/libprotobuf-mutator.cmake | 11 + contrib/CMakeLists.txt | 4 + contrib/libprotobuf-mutator | 1 + .../libprotobuf-mutator-cmake/CMakeLists.txt | 17 + src/Parsers/fuzzers/CMakeLists.txt | 5 + .../fuzzers/codegen_fuzzer/CMakeLists.txt | 13 + .../codegen_fuzzer/clickhouse-template.g | 121 + .../fuzzers/codegen_fuzzer/clickhouse.g | 524 ++++ .../codegen_fuzzer/codegen_select_fuzzer.cpp | 40 + src/Parsers/fuzzers/codegen_fuzzer/gen.py | 249 ++ src/Parsers/fuzzers/codegen_fuzzer/out.cpp | 2189 +++++++++++++++++ src/Parsers/fuzzers/codegen_fuzzer/out.proto | 519 ++++ src/Parsers/fuzzers/codegen_fuzzer/update.sh | 30 + 15 files changed, 3727 insertions(+) create mode 100644 cmake/find/libprotobuf-mutator.cmake create mode 160000 contrib/libprotobuf-mutator create mode 100644 contrib/libprotobuf-mutator-cmake/CMakeLists.txt create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/gen.py create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.cpp create mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.proto create mode 100755 src/Parsers/fuzzers/codegen_fuzzer/update.sh diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..5d226cddd29 100644 --- a/.gitmodules +++ b/.gitmodules @@ -249,3 +249,6 @@ [submodule "contrib/magic_enum"] path = contrib/magic_enum url = https://github.com/Neargye/magic_enum +[submodule "contrib/libprotobuf-mutator"] + path = contrib/libprotobuf-mutator + url = https://github.com/google/libprotobuf-mutator diff --git a/CMakeLists.txt b/CMakeLists.txt index 3f553c5c26d..843beec01c6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -562,6 +562,7 @@ include (cmake/find/cassandra.cmake) include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) +include (cmake/find/libprotobuf-mutator.cmake) set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/cmake/find/libprotobuf-mutator.cmake b/cmake/find/libprotobuf-mutator.cmake new file mode 100644 index 00000000000..8aa595230cd --- /dev/null +++ b/cmake/find/libprotobuf-mutator.cmake @@ -0,0 +1,11 @@ +option(USE_LIBPROTOBUF_MUTATOR "Enable libprotobuf-mutator" ${ENABLE_FUZZING}) + +if (NOT USE_LIBPROTOBUF_MUTATOR) + return() +endif() + +set(LibProtobufMutator_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator") + +if (NOT EXISTS "${LibProtobufMutator_SOURCE_DIR}/README.md") + message (ERROR "submodule contrib/libprotobuf-mutator is missing. to fix try run: \n git submodule update --init --recursive") +endif() diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 140cc0846ec..98231856aee 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -49,6 +49,10 @@ add_subdirectory (replxx-cmake) add_subdirectory (unixodbc-cmake) add_subdirectory (nanodbc-cmake) +if (ENABLE_FUZZING) + add_subdirectory (libprotobuf-mutator-cmake) +endif() + if (USE_YAML_CPP) add_subdirectory (yaml-cpp-cmake) endif() diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator new file mode 160000 index 00000000000..ffd86a32874 --- /dev/null +++ b/contrib/libprotobuf-mutator @@ -0,0 +1 @@ +Subproject commit ffd86a32874e5c08a143019aad1aaf0907294c9f diff --git a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt new file mode 100644 index 00000000000..93eafc85b7d --- /dev/null +++ b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt @@ -0,0 +1,17 @@ +set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator) +set(NO_FUZZING_FLAGS "-fno-sanitize=fuzzer -fsanitize-coverage=0") + +add_library(protobuf-mutator + ${LIBRARY_DIR}/src/libfuzzer/libfuzzer_macro.cc + ${LIBRARY_DIR}/src/libfuzzer/libfuzzer_mutator.cc + ${LIBRARY_DIR}/src/binary_format.cc + ${LIBRARY_DIR}/src/mutator.cc + ${LIBRARY_DIR}/src/text_format.cc + ${LIBRARY_DIR}/src/utf8_fix.cc) + +target_include_directories(protobuf-mutator BEFORE PRIVATE "${LIBRARY_DIR}") +# target_include_directories(protobuf-mutator BEFORE PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") + +target_link_libraries(protobuf-mutator ${PROTOBUF_LIBRARIES}) +set_target_properties(protobuf-mutator PROPERTIES + COMPILE_FLAGS "${NO_FUZZING_FLAGS}") diff --git a/src/Parsers/fuzzers/CMakeLists.txt b/src/Parsers/fuzzers/CMakeLists.txt index 0dd541e663f..2840dc72c0a 100644 --- a/src/Parsers/fuzzers/CMakeLists.txt +++ b/src/Parsers/fuzzers/CMakeLists.txt @@ -6,3 +6,8 @@ target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZ add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) + +string(REPLACE " -Werror" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") +add_subdirectory(codegen_fuzzer) +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror") + diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt new file mode 100644 index 00000000000..f55bb3b3fb9 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -0,0 +1,13 @@ +find_package(Protobuf REQUIRED) + +protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS "out.proto") +set(FUZZER_SRCS codegen_select_fuzzer.cpp out.cpp ${PROTO_SRCS} ${PROTO_HDRS}) + +set(CMAKE_INCLUDE_CURRENT_DIR TRUE) + +add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) + +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIRS}") +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") +target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) \ No newline at end of file diff --git a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g new file mode 100644 index 00000000000..79fd775b1da --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse-template.g @@ -0,0 +1,121 @@ +" "; +" "; +" "; +";"; + + +"(" $1 ")"; +"(" $1 ", " $2 ")"; +"(" $1 ", " $2 ", " $3 ")"; + +$1 ", " $2 ; +$1 ", " $2 ", " $3 ; +$1 ", " $2 ", " $3 ", " $4 ; +$1 ", " $2 ", " $3 ", " $4 ", " $5 ; + +"[" $1 ", " $2 "]"; +"[" $1 ", " $2 ", " $3 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 ", " $5 "]"; + +$0 "(" $1 ")"; +$0 "(" $1 ", " $2 ")"; +$0 "(" $1 ", " $2 ", " $3 ")"; + +$1 " as " $2 ; + + +// TODO: add more clickhouse specific stuff +"SELECT " $1 " FROM " $2 " WHERE " $3 ; +"SELECT " $1 " FROM " $2 " GROUP BY " $3 ; +"SELECT " $1 " FROM " $2 " SORT BY " $3 ; +"SELECT " $1 " FROM " $2 " LIMIT " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 ; +"SELECT " $1 " FROM " $2 " ARRAY JOIN " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " ON " $4 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " USING " $5 ; +"SELECT " $1 " INTO OUTFILE " $2 ; + +"WITH " $1 " AS " $2 ; + +"{" $1 ":" $2 "}"; +"[" $1 "," $2 "]"; +"[]"; + + +" x "; +"x"; +" `x` "; +"`x`"; + +" \"value\" "; +"\"value\""; +" 0 "; +"0"; +"1"; +"2"; +"123123123123123123"; +"182374019873401982734091873420923123123123123123"; +"1e-1"; +"1.1"; +"\"\""; +" '../../../../../../../../../etc/passwd' "; + +"/"; +"="; +"=="; +"!="; +"<>"; +"<"; +"<="; +">"; +">="; +"<<"; +"|<<"; +"&"; +"|"; +"||"; +"<|"; +"|>"; +"+"; +"-"; +"~"; +"*"; +"/"; +"\\"; +"%"; +""; +"."; +","; +","; +","; +","; +","; +","; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"?"; +":"; +"@"; +"@@"; +"$"; +"\""; +"`"; +"{"; +"}"; +"^"; +"::"; +"->"; +"]"; +"["; + diff --git a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g new file mode 100644 index 00000000000..edd5acf513d --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g @@ -0,0 +1,524 @@ +" "; +" "; +" "; +";"; + + +"(" $1 ")"; +"(" $1 ", " $2 ")"; +"(" $1 ", " $2 ", " $3 ")"; + +$1 ", " $2 ; +$1 ", " $2 ", " $3 ; +$1 ", " $2 ", " $3 ", " $4 ; +$1 ", " $2 ", " $3 ", " $4 ", " $5 ; + +"[" $1 ", " $2 "]"; +"[" $1 ", " $2 ", " $3 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 "]"; +"[" $1 ", " $2 ", " $3 ", " $4 ", " $5 "]"; + +$0 "(" $1 ")"; +$0 "(" $1 ", " $2 ")"; +$0 "(" $1 ", " $2 ", " $3 ")"; + +$1 " as " $2 ; + + +// TODO: add more clickhouse specific stuff +"SELECT " $1 " FROM " $2 " WHERE " $3 ; +"SELECT " $1 " FROM " $2 " GROUP BY " $3 ; +"SELECT " $1 " FROM " $2 " SORT BY " $3 ; +"SELECT " $1 " FROM " $2 " LIMIT " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 ; +"SELECT " $1 " FROM " $2 " ARRAY JOIN " $3 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " ON " $4 ; +"SELECT " $1 " FROM " $2 " JOIN " $3 " USING " $5 ; +"SELECT " $1 " INTO OUTFILE " $2 ; + +"WITH " $1 " AS " $2 ; + +"{" $1 ":" $2 "}"; +"[" $1 "," $2 "]"; +"[]"; + + +" x "; +"x"; +" `x` "; +"`x`"; + +" \"value\" "; +"\"value\""; +" 0 "; +"0"; +"1"; +"2"; +"123123123123123123"; +"182374019873401982734091873420923123123123123123"; +"1e-1"; +"1.1"; +"\"\""; +" '../../../../../../../../../etc/passwd' "; + +"/"; +"="; +"=="; +"!="; +"<>"; +"<"; +"<="; +">"; +">="; +"<<"; +"|<<"; +"&"; +"|"; +"||"; +"<|"; +"|>"; +"+"; +"-"; +"~"; +"*"; +"/"; +"\\"; +"%"; +""; +"."; +","; +","; +","; +","; +","; +","; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"("; +")"; +"?"; +":"; +"@"; +"@@"; +"$"; +"\""; +"`"; +"{"; +"}"; +"^"; +"::"; +"->"; +"]"; +"["; + +" ADD "; +" ADD COLUMN "; +" ADD CONSTRAINT "; +" ADD INDEX "; +" AFTER "; +" AggregateFunction "; +" aggThrow "; +" ALL "; +" ALTER LIVE VIEW "; +" ALTER TABLE "; +" and "; +" ANTI "; +" any "; +" anyHeavy "; +" anyLast "; +" argMax "; +" argMin "; +" array "; +" Array "; +" arrayAll "; +" arrayAUC "; +" arrayCompact "; +" arrayConcat "; +" arrayCount "; +" arrayCumSum "; +" arrayCumSumNonNegative "; +" arrayDifference "; +" arrayDistinct "; +" arrayElement "; +" arrayEnumerate "; +" arrayEnumerateDense "; +" arrayEnumerateDenseRanked "; +" arrayEnumerateUniq "; +" arrayEnumerateUniqRanked "; +" arrayExists "; +" arrayFill "; +" arrayFilter "; +" arrayFirst "; +" arrayFirstIndex "; +" arrayFlatten "; +" arrayIntersect "; +" arrayJoin "; +" ARRAY JOIN "; +" arrayMap "; +" arrayPopBack "; +" arrayPopFront "; +" arrayPushBack "; +" arrayPushFront "; +" arrayReduce "; +" arrayReduceInRanges "; +" arrayResize "; +" arrayReverse "; +" arrayReverseFill "; +" arrayReverseSort "; +" arrayReverseSplit "; +" arraySlice "; +" arraySort "; +" arraySplit "; +" arraySum "; +" arrayUniq "; +" arrayWithConstant "; +" arrayZip "; +" AS "; +" ASC "; +" ASCENDING "; +" ASOF "; +" AST "; +" ATTACH "; +" ATTACH PART "; +" ATTACH PARTITION "; +" avg "; +" avgWeighted "; +" basename "; +" BETWEEN "; +" BOTH "; +" boundingRatio "; +" BY "; +" CAST "; +" categoricalInformationValue "; +" CHECK "; +" CHECK TABLE "; +" CLEAR COLUMN "; +" CLEAR INDEX "; +" COLLATE "; +" COLUMNS "; +" COMMENT COLUMN "; +" CONSTRAINT "; +" corr "; +" corrStable "; +" count "; +" countEqual "; +" covarPop "; +" covarPopStable "; +" covarSamp "; +" covarSampStable "; +" CREATE "; +" CROSS "; +" CUBE "; +" cutFragment "; +" cutQueryString "; +" cutQueryStringAndFragment "; +" cutToFirstSignificantSubdomain "; +" cutURLParameter "; +" cutWWW "; +" D "; +" DATABASE "; +" DATABASES "; +" Date "; +" DATE "; +" DATE_ADD "; +" DATEADD "; +" DATE_DIFF "; +" DATEDIFF "; +" DATE_SUB "; +" DATESUB "; +" DateTime "; +" DateTime64 "; +" DAY "; +" DD "; +" Decimal "; +" Decimal128 "; +" Decimal32 "; +" Decimal64 "; +" decodeURLComponent "; +" DEDUPLICATE "; +" DELETE "; +" DELETE WHERE "; +" DESC "; +" DESCENDING "; +" DESCRIBE "; +" DETACH "; +" DETACH PARTITION "; +" DICTIONARIES "; +" DICTIONARY "; +" DISTINCT "; +" domain "; +" domainWithoutWWW "; +" DROP "; +" DROP COLUMN "; +" DROP CONSTRAINT "; +" DROP DETACHED PART "; +" DROP DETACHED PARTITION "; +" DROP INDEX "; +" DROP PARTITION "; +" emptyArrayToSingle "; +" ENGINE "; +" entropy "; +" Enum "; +" Enum16 "; +" Enum8 "; +" EVENTS "; +" EXCHANGE TABLES "; +" EXISTS "; +" EXTRACT "; +" extractURLParameter "; +" extractURLParameterNames "; +" extractURLParameters "; +" FETCH PARTITION "; +" FETCH PART "; +" FINAL "; +" FIRST "; +" firstSignificantSubdomain "; +" FixedString "; +" Float32 "; +" Float64 "; +" FOR "; +" ForEach "; +" FORMAT "; +" fragment "; +" FREEZE "; +" FROM "; +" FULL "; +" FUNCTION "; +" __getScalar "; +" GLOBAL "; +" GRANULARITY "; +" groupArray "; +" groupArrayInsertAt "; +" groupArrayMovingAvg "; +" groupArrayMovingSum "; +" groupArraySample "; +" groupBitAnd "; +" groupBitmap "; +" groupBitmapAnd "; +" groupBitmapOr "; +" groupBitmapXor "; +" groupBitOr "; +" groupBitXor "; +" GROUP BY "; +" groupUniqArray "; +" has "; +" hasAll "; +" hasAny "; +" HAVING "; +" HH "; +" histogram "; +" HOUR "; +" ID "; +" if "; +" IF EXISTS "; +" IF NOT EXISTS "; +" IN "; +" INDEX "; +" indexOf "; +" INNER "; +" IN PARTITION "; +" INSERT INTO "; +" Int16 "; +" Int32 "; +" Int64 "; +" Int8 "; +" INTERVAL "; +" IntervalDay "; +" IntervalHour "; +" IntervalMinute "; +" IntervalMonth "; +" IntervalQuarter "; +" IntervalSecond "; +" IntervalWeek "; +" IntervalYear "; +" INTO OUTFILE "; +" JOIN "; +" kurtPop "; +" kurtSamp "; +" LAST "; +" LAYOUT "; +" LEADING "; +" LEFT "; +" LEFT ARRAY JOIN "; +" length "; +" LIFETIME "; +" LIKE "; +" LIMIT "; +" LIVE "; +" LOCAL "; +" LowCardinality "; +" LTRIM "; +" M "; +" MATERIALIZED "; +" MATERIALIZE INDEX "; +" MATERIALIZE TTL "; +" max "; +" maxIntersections "; +" maxIntersectionsPosition "; +" Merge "; +" MI "; +" min "; +" MINUTE "; +" MM "; +" MODIFY "; +" MODIFY COLUMN "; +" MODIFY ORDER BY "; +" MODIFY QUERY "; +" MODIFY SETTING "; +" MODIFY TTL "; +" MONTH "; +" MOVE PART "; +" MOVE PARTITION "; +" movingXXX "; +" N "; +" NAME "; +" Nested "; +" NO DELAY "; +" NONE "; +" not "; +" nothing "; +" Nothing "; +" Null "; +" Nullable "; +" NULLS "; +" OFFSET "; +" ON "; +" ONLY "; +" OPTIMIZE TABLE "; +" ORDER BY "; +" OR REPLACE "; +" OUTER "; +" PARTITION "; +" PARTITION BY "; +" path "; +" pathFull "; +" POPULATE "; +" PREWHERE "; +" PRIMARY KEY "; +" protocol "; +" Q "; +" QQ "; +" QUARTER "; +" queryString "; +" queryStringAndFragment "; +" range "; +" REFRESH "; +" RENAME COLUMN "; +" RENAME TABLE "; +" REPLACE PARTITION "; +" Resample "; +" RESUME "; +" retention "; +" RIGHT "; +" ROLLUP "; +" RTRIM "; +" S "; +" SAMPLE "; +" SAMPLE BY "; +" SECOND "; +" SELECT "; +" SEMI "; +" sequenceCount "; +" sequenceMatch "; +" SET "; +" SETTINGS "; +" SHOW "; +" SHOW PROCESSLIST "; +" simpleLinearRegression "; +" skewPop "; +" skewSamp "; +" SOURCE "; +" SQL_TSI_DAY "; +" SQL_TSI_HOUR "; +" SQL_TSI_MINUTE "; +" SQL_TSI_MONTH "; +" SQL_TSI_QUARTER "; +" SQL_TSI_SECOND "; +" SQL_TSI_WEEK "; +" SQL_TSI_YEAR "; +" SS "; +" State "; +" stddevPop "; +" stddevPopStable "; +" stddevSamp "; +" stddevSampStable "; +" STEP "; +" stochasticLinearRegression "; +" stochasticLogisticRegression "; +" String "; +" SUBSTRING "; +" sum "; +" sumKahan "; +" sumMap "; +" sumMapFiltered "; +" sumMapFilteredWithOverflow "; +" sumMapWithOverflow "; +" sumWithOverflow "; +" SUSPEND "; +" TABLE "; +" TABLES "; +" TEMPORARY "; +" TIMESTAMP "; +" TIMESTAMP_ADD "; +" TIMESTAMPADD "; +" TIMESTAMP_DIFF "; +" TIMESTAMPDIFF "; +" TIMESTAMP_SUB "; +" TIMESTAMPSUB "; +" TO "; +" TO DISK "; +" TOP "; +" topK "; +" topKWeighted "; +" topLevelDomain "; +" TO TABLE "; +" TOTALS "; +" TO VOLUME "; +" TRAILING "; +" TRIM "; +" TRUNCATE "; +" TTL "; +" Tuple "; +" TYPE "; +" UInt16 "; +" UInt32 "; +" UInt64 "; +" UInt8 "; +" uniq "; +" uniqCombined "; +" uniqCombined64 "; +" uniqExact "; +" uniqHLL12 "; +" uniqUpTo "; +" UPDATE "; +" URLHierarchy "; +" URLPathHierarchy "; +" USE "; +" USING "; +" UUID "; +" VALUES "; +" varPop "; +" varPopStable "; +" varSamp "; +" varSampStable "; +" VIEW "; +" WATCH "; +" WEEK "; +" WHERE "; +" windowFunnel "; +" WITH "; +" WITH FILL "; +" WITH TIES "; +" WK "; +" WW "; +" YEAR "; +" YY "; +" YYYY "; diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp new file mode 100644 index 00000000000..a68fac4f585 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -0,0 +1,40 @@ + +#include +#include + +#include +#include +#include +#include + +#include + +#include "out.pb.h" + + +void GenerateSentence(const Sentence&, std::string &, int); + + +DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) { + static std::string input; + input.reserve(4096); + + GenerateSentence(main, input, 0); + if (input.size()) { + + std::cout << input << std::endl; + + DB::ParserQueryWithOutput parser(input.data() + input.size()); + try { + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + + DB::WriteBufferFromOStream out(std::cerr, 4096); + DB::formatAST(*ast, out); + std::cerr << std::endl; + } catch (...) { + + } + + input.clear(); + } +} diff --git a/src/Parsers/fuzzers/codegen_fuzzer/gen.py b/src/Parsers/fuzzers/codegen_fuzzer/gen.py new file mode 100644 index 00000000000..e96bc6ae9f6 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/gen.py @@ -0,0 +1,249 @@ +#!/usr/bin/env python3 + +import sys +import string + + +TOKEN_TEXT = 1 +TOKEN_VAR = 2 + +TOKEN_COLON = ':' +TOKEN_SEMI = ';' +TOKEN_OR = '|' +TOKEN_QUESTIONMARK = '?' +TOKEN_ROUND_BRACKET_OPEN = '(' +TOKEN_ROUND_BRACKET_CLOSE = ')' +TOKEN_ASTERISK = '*' +TOKEN_SLASH = '/' + + + + +class TextValue: + def __init__(self, t): + self.t = t + self.slug = None + + def get_slug(self): + if self.slug is not None: + return self.slug + slug = '' + for c in self.t: + slug += c if c in string.ascii_letters else '_' + self.slug = slug + return slug + + def get_name(self): + return f"TextValue_{self.get_slug()}" + + def __repr__(self): + return f"TextValue(\"{self.t}\")" + + +class Var: + def __init__(self, id_): + self.id_ = id_ + + def __repr__(self): + return f"Var({self.id_})" + + +class Parser: + def __init__(self): + self.chains = [] + self.text = None + self.col = 0 + self.line = 1 + self.t = None + self.var_id = -1 + self.cur_tok = None + self.includes = [] + + self.proto = '' + self.cpp = '' + + def parse_file(self, filename): + with open(filename) as f: + self.text = f.read() + + while self.parse_statement() is not None: + pass + + def add_include(self, filename): + self.includes.append(filename) + + def get_next_token(self): + self.skip_ws() + + if not len(self.text): + return None + + if self.text[0] == '"': + return self.parse_txt_value() + + if self.text[0] == '$': + return self.parse_var_value() + + c, self.text = self.text[0], self.text[1:] + self.cur_tok = c + return c + + def parse_var_value(self): + i = self.text.find(' ') + + id_, self.text = self.text[1:i], self.text[i+1:] + self.var_id = int(id_) + self.cur_tok = TOKEN_VAR + return TOKEN_VAR + + def parse_txt_value(self): + if self.text[0] != '"': + raise Exception("parse_txt_value: expected quote at the start") + + self.t = '' + self.text = self.text[1:] + + while self.text[0] != '"': + if self.text[0] == '\\': + if self.text[1] == 'x': + self.t += self.text[:4] + self.text = self.text[4:] + elif self.text[1] in 'nt\\"': + self.t += self.text[:2] + self.text = self.text[2:] + else: + raise Exception(f"parse_txt_value: unknown symbol {self.text[0]}") + else: + c, self.text = self.text[0], self.text[1:] + self.t += c + + self.text = self.text[1:] + self.cur_tok = TOKEN_TEXT + return TOKEN_TEXT + + def skip_ws(self): + while self.text and self.text[0] in string.whitespace: + if self.text[0] == '\n': + self.line += 1 + self.col = 0 + self.text = self.text[1:] + self.col += 1 + if not self.text: + return None + return True + + def skip_line(self): + self.line += 1 + index = self.text.find('\n') + self.text = self.text[index:] + + + def parse_statement(self): + if self.skip_ws() is None: + return None + + self.get_next_token() + if self.cur_tok == TOKEN_SLASH: + self.skip_line() + return TOKEN_SLASH + + chain = [] + while self.cur_tok != TOKEN_SEMI: + if self.cur_tok == TOKEN_TEXT: + chain.append(TextValue(self.t)) + elif self.cur_tok == TOKEN_VAR: + chain.append(Var(self.var_id)) + else: + self.fatal_parsing_error(f"unexpected token {self.tok}") + self.get_next_token() + + if not chain: + self.fatal_parsing_error("empty chains are not allowed") + self.chains.append(chain) + return True + + def generate(self): + self.proto = 'syntax = "proto3";\n\n' + self.cpp = '#include \n#include \n#include \n\n#include \n\n' + + for incl_file in self.includes: + self.cpp += f'#include "{incl_file}"\n' + self.cpp += '\n' + + self.proto += 'message Word {\n' + self.proto += '\tenum Value {\n' + + self.cpp += 'void GenerateWord(const Word&, std::string&, int);\n\n' + + self.cpp += 'void GenerateSentence(const Sentence& stc, std::string &s, int depth) {\n' + self.cpp += '\tfor (int i = 0; i < stc.words_size(); i++ ) {\n' + self.cpp += '\t\tGenerateWord(stc.words(i), s, ++depth);\n' + self.cpp += '\t}\n' + self.cpp += '}\n' + + self.cpp += 'void GenerateWord(const Word& word, std::string &s, int depth) {\n' + + self.cpp += '\tif (depth > 5) return;\n\n' + self.cpp += '\tswitch (word.value()) {\n' + + for idx, chain in enumerate(self.chains): + self.proto += f'\t\tvalue_{idx} = {idx};\n' + + self.cpp += f'\t\tcase {idx}: {{\n' + num_var = 0 + for item in chain: + if isinstance(item, TextValue): + self.cpp += f'\t\t\ts += "{item.t}";\n' + elif isinstance(item, Var): + self.cpp += f'\t\t\tif (word.inner().words_size() > {num_var})\t\t\t\tGenerateWord(word.inner().words({num_var}), s, ++depth);\n' + num_var += 1 + else: + raise Exception("unknown token met during generation") + self.cpp += '\t\t\tbreak;\n\t\t}\n' + self.cpp += '\t\tdefault: break;\n' + + self.cpp += '\t}\n' + + self.proto += '\t}\n' + self.proto += '\tValue value = 1;\n' + self.proto += '\tSentence inner = 2;\n' + self.proto += '}\nmessage Sentence {\n\trepeated Word words = 1;\n}' + + self.cpp += '}\n' + return self.cpp, self.proto + + def fatal_parsing_error(self, msg): + print(f"Line: {self.line}, Col: {self.col}") + raise Exception(f"fatal error during parsing. {msg}") + + +def main(args): + input_file, outfile_cpp, outfile_proto = args + + if not outfile_proto.endswith('.proto'): + raise Exception("outfile_proto (argv[3]) should end with `.proto`") + + include_filename = outfile_proto[:-6] + ".pb.h" + + p = Parser() + p.add_include(include_filename) + p.parse_file(input_file) + + cpp, proto = p.generate() + + proto = proto.replace('\t', ' ' * 4) + cpp = cpp.replace('\t', ' ' * 4) + + with open(outfile_cpp, 'w') as f: + f.write(cpp) + + with open(outfile_proto, 'w') as f: + f.write(proto) + + +if __name__ == '__main__': + if len(sys.argv) < 3: + print(f"Usage {sys.argv[0]} ") + sys.exit(1) + main(sys.argv[1:]) + diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.cpp b/src/Parsers/fuzzers/codegen_fuzzer/out.cpp new file mode 100644 index 00000000000..effe6e7821b --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/out.cpp @@ -0,0 +1,2189 @@ +#include +#include +#include + +#include + +#include "out.pb.h" + +void GenerateWord(const Word&, std::string&, int); + +void GenerateSentence(const Sentence& stc, std::string &s, int depth) { + for (int i = 0; i < stc.words_size(); i++ ) { + GenerateWord(stc.words(i), s, ++depth); + } +} +void GenerateWord(const Word& word, std::string &s, int depth) { + if (depth > 5) return; + + switch (word.value()) { + case 0: { + s += " "; + break; + } + case 1: { + s += " "; + break; + } + case 2: { + s += " "; + break; + } + case 3: { + s += ";"; + break; + } + case 4: { + s += "("; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ")"; + break; + } + case 5: { + s += "("; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ")"; + break; + } + case 6: { + s += "("; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ")"; + break; + } + case 7: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 8: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 9: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + break; + } + case 10: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += ", "; + if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); + break; + } + case 11: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += "]"; + break; + } + case 12: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += "]"; + break; + } + case 13: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += "]"; + break; + } + case 14: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ", "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += ", "; + if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); + s += "]"; + break; + } + case 15: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += "("; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ")"; + break; + } + case 16: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += "("; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ")"; + break; + } + case 17: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += "("; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += ", "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += ", "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + s += ")"; + break; + } + case 18: { + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " as "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 19: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " WHERE "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 20: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " GROUP BY "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 21: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " SORT BY "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 22: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " LIMIT "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 23: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 24: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " ARRAY JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + break; + } + case 25: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += " ON "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + break; + } + case 26: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " FROM "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += " JOIN "; + if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); + s += " USING "; + if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); + break; + } + case 27: { + s += "SELECT "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " INTO OUTFILE "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 28: { + s += "WITH "; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += " AS "; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + break; + } + case 29: { + s += "{"; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ":"; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += "}"; + break; + } + case 30: { + s += "["; + if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); + s += ","; + if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); + s += "]"; + break; + } + case 31: { + s += "[]"; + break; + } + case 32: { + s += " x "; + break; + } + case 33: { + s += "x"; + break; + } + case 34: { + s += " `x` "; + break; + } + case 35: { + s += "`x`"; + break; + } + case 36: { + s += " \"value\" "; + break; + } + case 37: { + s += "\"value\""; + break; + } + case 38: { + s += " 0 "; + break; + } + case 39: { + s += "0"; + break; + } + case 40: { + s += "1"; + break; + } + case 41: { + s += "2"; + break; + } + case 42: { + s += "123123123123123123"; + break; + } + case 43: { + s += "182374019873401982734091873420923123123123123123"; + break; + } + case 44: { + s += "1e-1"; + break; + } + case 45: { + s += "1.1"; + break; + } + case 46: { + s += "\"\""; + break; + } + case 47: { + s += " '../../../../../../../../../etc/passwd' "; + break; + } + case 48: { + s += "/"; + break; + } + case 49: { + s += "="; + break; + } + case 50: { + s += "=="; + break; + } + case 51: { + s += "!="; + break; + } + case 52: { + s += "<>"; + break; + } + case 53: { + s += "<"; + break; + } + case 54: { + s += "<="; + break; + } + case 55: { + s += ">"; + break; + } + case 56: { + s += ">="; + break; + } + case 57: { + s += "<<"; + break; + } + case 58: { + s += "|<<"; + break; + } + case 59: { + s += "&"; + break; + } + case 60: { + s += "|"; + break; + } + case 61: { + s += "||"; + break; + } + case 62: { + s += "<|"; + break; + } + case 63: { + s += "|>"; + break; + } + case 64: { + s += "+"; + break; + } + case 65: { + s += "-"; + break; + } + case 66: { + s += "~"; + break; + } + case 67: { + s += "*"; + break; + } + case 68: { + s += "/"; + break; + } + case 69: { + s += "\\"; + break; + } + case 70: { + s += "%"; + break; + } + case 71: { + s += ""; + break; + } + case 72: { + s += "."; + break; + } + case 73: { + s += ","; + break; + } + case 74: { + s += ","; + break; + } + case 75: { + s += ","; + break; + } + case 76: { + s += ","; + break; + } + case 77: { + s += ","; + break; + } + case 78: { + s += ","; + break; + } + case 79: { + s += "("; + break; + } + case 80: { + s += ")"; + break; + } + case 81: { + s += "("; + break; + } + case 82: { + s += ")"; + break; + } + case 83: { + s += "("; + break; + } + case 84: { + s += ")"; + break; + } + case 85: { + s += "("; + break; + } + case 86: { + s += ")"; + break; + } + case 87: { + s += "("; + break; + } + case 88: { + s += ")"; + break; + } + case 89: { + s += "("; + break; + } + case 90: { + s += ")"; + break; + } + case 91: { + s += "?"; + break; + } + case 92: { + s += ":"; + break; + } + case 93: { + s += "@"; + break; + } + case 94: { + s += "@@"; + break; + } + case 95: { + s += "$"; + break; + } + case 96: { + s += "\""; + break; + } + case 97: { + s += "`"; + break; + } + case 98: { + s += "{"; + break; + } + case 99: { + s += "}"; + break; + } + case 100: { + s += "^"; + break; + } + case 101: { + s += "::"; + break; + } + case 102: { + s += "->"; + break; + } + case 103: { + s += "]"; + break; + } + case 104: { + s += "["; + break; + } + case 105: { + s += " ADD "; + break; + } + case 106: { + s += " ADD COLUMN "; + break; + } + case 107: { + s += " ADD CONSTRAINT "; + break; + } + case 108: { + s += " ADD INDEX "; + break; + } + case 109: { + s += " AFTER "; + break; + } + case 110: { + s += " AggregateFunction "; + break; + } + case 111: { + s += " aggThrow "; + break; + } + case 112: { + s += " ALL "; + break; + } + case 113: { + s += " ALTER LIVE VIEW "; + break; + } + case 114: { + s += " ALTER TABLE "; + break; + } + case 115: { + s += " and "; + break; + } + case 116: { + s += " ANTI "; + break; + } + case 117: { + s += " any "; + break; + } + case 118: { + s += " anyHeavy "; + break; + } + case 119: { + s += " anyLast "; + break; + } + case 120: { + s += " argMax "; + break; + } + case 121: { + s += " argMin "; + break; + } + case 122: { + s += " array "; + break; + } + case 123: { + s += " Array "; + break; + } + case 124: { + s += " arrayAll "; + break; + } + case 125: { + s += " arrayAUC "; + break; + } + case 126: { + s += " arrayCompact "; + break; + } + case 127: { + s += " arrayConcat "; + break; + } + case 128: { + s += " arrayCount "; + break; + } + case 129: { + s += " arrayCumSum "; + break; + } + case 130: { + s += " arrayCumSumNonNegative "; + break; + } + case 131: { + s += " arrayDifference "; + break; + } + case 132: { + s += " arrayDistinct "; + break; + } + case 133: { + s += " arrayElement "; + break; + } + case 134: { + s += " arrayEnumerate "; + break; + } + case 135: { + s += " arrayEnumerateDense "; + break; + } + case 136: { + s += " arrayEnumerateDenseRanked "; + break; + } + case 137: { + s += " arrayEnumerateUniq "; + break; + } + case 138: { + s += " arrayEnumerateUniqRanked "; + break; + } + case 139: { + s += " arrayExists "; + break; + } + case 140: { + s += " arrayFill "; + break; + } + case 141: { + s += " arrayFilter "; + break; + } + case 142: { + s += " arrayFirst "; + break; + } + case 143: { + s += " arrayFirstIndex "; + break; + } + case 144: { + s += " arrayFlatten "; + break; + } + case 145: { + s += " arrayIntersect "; + break; + } + case 146: { + s += " arrayJoin "; + break; + } + case 147: { + s += " ARRAY JOIN "; + break; + } + case 148: { + s += " arrayMap "; + break; + } + case 149: { + s += " arrayPopBack "; + break; + } + case 150: { + s += " arrayPopFront "; + break; + } + case 151: { + s += " arrayPushBack "; + break; + } + case 152: { + s += " arrayPushFront "; + break; + } + case 153: { + s += " arrayReduce "; + break; + } + case 154: { + s += " arrayReduceInRanges "; + break; + } + case 155: { + s += " arrayResize "; + break; + } + case 156: { + s += " arrayReverse "; + break; + } + case 157: { + s += " arrayReverseFill "; + break; + } + case 158: { + s += " arrayReverseSort "; + break; + } + case 159: { + s += " arrayReverseSplit "; + break; + } + case 160: { + s += " arraySlice "; + break; + } + case 161: { + s += " arraySort "; + break; + } + case 162: { + s += " arraySplit "; + break; + } + case 163: { + s += " arraySum "; + break; + } + case 164: { + s += " arrayUniq "; + break; + } + case 165: { + s += " arrayWithConstant "; + break; + } + case 166: { + s += " arrayZip "; + break; + } + case 167: { + s += " AS "; + break; + } + case 168: { + s += " ASC "; + break; + } + case 169: { + s += " ASCENDING "; + break; + } + case 170: { + s += " ASOF "; + break; + } + case 171: { + s += " AST "; + break; + } + case 172: { + s += " ATTACH "; + break; + } + case 173: { + s += " ATTACH PART "; + break; + } + case 174: { + s += " ATTACH PARTITION "; + break; + } + case 175: { + s += " avg "; + break; + } + case 176: { + s += " avgWeighted "; + break; + } + case 177: { + s += " basename "; + break; + } + case 178: { + s += " BETWEEN "; + break; + } + case 179: { + s += " BOTH "; + break; + } + case 180: { + s += " boundingRatio "; + break; + } + case 181: { + s += " BY "; + break; + } + case 182: { + s += " CAST "; + break; + } + case 183: { + s += " categoricalInformationValue "; + break; + } + case 184: { + s += " CHECK "; + break; + } + case 185: { + s += " CHECK TABLE "; + break; + } + case 186: { + s += " CLEAR COLUMN "; + break; + } + case 187: { + s += " CLEAR INDEX "; + break; + } + case 188: { + s += " COLLATE "; + break; + } + case 189: { + s += " COLUMNS "; + break; + } + case 190: { + s += " COMMENT COLUMN "; + break; + } + case 191: { + s += " CONSTRAINT "; + break; + } + case 192: { + s += " corr "; + break; + } + case 193: { + s += " corrStable "; + break; + } + case 194: { + s += " count "; + break; + } + case 195: { + s += " countEqual "; + break; + } + case 196: { + s += " covarPop "; + break; + } + case 197: { + s += " covarPopStable "; + break; + } + case 198: { + s += " covarSamp "; + break; + } + case 199: { + s += " covarSampStable "; + break; + } + case 200: { + s += " CREATE "; + break; + } + case 201: { + s += " CROSS "; + break; + } + case 202: { + s += " CUBE "; + break; + } + case 203: { + s += " cutFragment "; + break; + } + case 204: { + s += " cutQueryString "; + break; + } + case 205: { + s += " cutQueryStringAndFragment "; + break; + } + case 206: { + s += " cutToFirstSignificantSubdomain "; + break; + } + case 207: { + s += " cutURLParameter "; + break; + } + case 208: { + s += " cutWWW "; + break; + } + case 209: { + s += " D "; + break; + } + case 210: { + s += " DATABASE "; + break; + } + case 211: { + s += " DATABASES "; + break; + } + case 212: { + s += " Date "; + break; + } + case 213: { + s += " DATE "; + break; + } + case 214: { + s += " DATE_ADD "; + break; + } + case 215: { + s += " DATEADD "; + break; + } + case 216: { + s += " DATE_DIFF "; + break; + } + case 217: { + s += " DATEDIFF "; + break; + } + case 218: { + s += " DATE_SUB "; + break; + } + case 219: { + s += " DATESUB "; + break; + } + case 220: { + s += " DateTime "; + break; + } + case 221: { + s += " DateTime64 "; + break; + } + case 222: { + s += " DAY "; + break; + } + case 223: { + s += " DD "; + break; + } + case 224: { + s += " Decimal "; + break; + } + case 225: { + s += " Decimal128 "; + break; + } + case 226: { + s += " Decimal32 "; + break; + } + case 227: { + s += " Decimal64 "; + break; + } + case 228: { + s += " decodeURLComponent "; + break; + } + case 229: { + s += " DEDUPLICATE "; + break; + } + case 230: { + s += " DELETE "; + break; + } + case 231: { + s += " DELETE WHERE "; + break; + } + case 232: { + s += " DESC "; + break; + } + case 233: { + s += " DESCENDING "; + break; + } + case 234: { + s += " DESCRIBE "; + break; + } + case 235: { + s += " DETACH "; + break; + } + case 236: { + s += " DETACH PARTITION "; + break; + } + case 237: { + s += " DICTIONARIES "; + break; + } + case 238: { + s += " DICTIONARY "; + break; + } + case 239: { + s += " DISTINCT "; + break; + } + case 240: { + s += " domain "; + break; + } + case 241: { + s += " domainWithoutWWW "; + break; + } + case 242: { + s += " DROP "; + break; + } + case 243: { + s += " DROP COLUMN "; + break; + } + case 244: { + s += " DROP CONSTRAINT "; + break; + } + case 245: { + s += " DROP DETACHED PART "; + break; + } + case 246: { + s += " DROP DETACHED PARTITION "; + break; + } + case 247: { + s += " DROP INDEX "; + break; + } + case 248: { + s += " DROP PARTITION "; + break; + } + case 249: { + s += " emptyArrayToSingle "; + break; + } + case 250: { + s += " ENGINE "; + break; + } + case 251: { + s += " entropy "; + break; + } + case 252: { + s += " Enum "; + break; + } + case 253: { + s += " Enum16 "; + break; + } + case 254: { + s += " Enum8 "; + break; + } + case 255: { + s += " EVENTS "; + break; + } + case 256: { + s += " EXCHANGE TABLES "; + break; + } + case 257: { + s += " EXISTS "; + break; + } + case 258: { + s += " EXTRACT "; + break; + } + case 259: { + s += " extractURLParameter "; + break; + } + case 260: { + s += " extractURLParameterNames "; + break; + } + case 261: { + s += " extractURLParameters "; + break; + } + case 262: { + s += " FETCH PARTITION "; + break; + } + case 263: { + s += " FETCH PART "; + break; + } + case 264: { + s += " FINAL "; + break; + } + case 265: { + s += " FIRST "; + break; + } + case 266: { + s += " firstSignificantSubdomain "; + break; + } + case 267: { + s += " FixedString "; + break; + } + case 268: { + s += " Float32 "; + break; + } + case 269: { + s += " Float64 "; + break; + } + case 270: { + s += " FOR "; + break; + } + case 271: { + s += " ForEach "; + break; + } + case 272: { + s += " FORMAT "; + break; + } + case 273: { + s += " fragment "; + break; + } + case 274: { + s += " FREEZE "; + break; + } + case 275: { + s += " FROM "; + break; + } + case 276: { + s += " FULL "; + break; + } + case 277: { + s += " FUNCTION "; + break; + } + case 278: { + s += " __getScalar "; + break; + } + case 279: { + s += " GLOBAL "; + break; + } + case 280: { + s += " GRANULARITY "; + break; + } + case 281: { + s += " groupArray "; + break; + } + case 282: { + s += " groupArrayInsertAt "; + break; + } + case 283: { + s += " groupArrayMovingAvg "; + break; + } + case 284: { + s += " groupArrayMovingSum "; + break; + } + case 285: { + s += " groupArraySample "; + break; + } + case 286: { + s += " groupBitAnd "; + break; + } + case 287: { + s += " groupBitmap "; + break; + } + case 288: { + s += " groupBitmapAnd "; + break; + } + case 289: { + s += " groupBitmapOr "; + break; + } + case 290: { + s += " groupBitmapXor "; + break; + } + case 291: { + s += " groupBitOr "; + break; + } + case 292: { + s += " groupBitXor "; + break; + } + case 293: { + s += " GROUP BY "; + break; + } + case 294: { + s += " groupUniqArray "; + break; + } + case 295: { + s += " has "; + break; + } + case 296: { + s += " hasAll "; + break; + } + case 297: { + s += " hasAny "; + break; + } + case 298: { + s += " HAVING "; + break; + } + case 299: { + s += " HH "; + break; + } + case 300: { + s += " histogram "; + break; + } + case 301: { + s += " HOUR "; + break; + } + case 302: { + s += " ID "; + break; + } + case 303: { + s += " if "; + break; + } + case 304: { + s += " IF EXISTS "; + break; + } + case 305: { + s += " IF NOT EXISTS "; + break; + } + case 306: { + s += " IN "; + break; + } + case 307: { + s += " INDEX "; + break; + } + case 308: { + s += " indexOf "; + break; + } + case 309: { + s += " INNER "; + break; + } + case 310: { + s += " IN PARTITION "; + break; + } + case 311: { + s += " INSERT INTO "; + break; + } + case 312: { + s += " Int16 "; + break; + } + case 313: { + s += " Int32 "; + break; + } + case 314: { + s += " Int64 "; + break; + } + case 315: { + s += " Int8 "; + break; + } + case 316: { + s += " INTERVAL "; + break; + } + case 317: { + s += " IntervalDay "; + break; + } + case 318: { + s += " IntervalHour "; + break; + } + case 319: { + s += " IntervalMinute "; + break; + } + case 320: { + s += " IntervalMonth "; + break; + } + case 321: { + s += " IntervalQuarter "; + break; + } + case 322: { + s += " IntervalSecond "; + break; + } + case 323: { + s += " IntervalWeek "; + break; + } + case 324: { + s += " IntervalYear "; + break; + } + case 325: { + s += " INTO OUTFILE "; + break; + } + case 326: { + s += " JOIN "; + break; + } + case 327: { + s += " kurtPop "; + break; + } + case 328: { + s += " kurtSamp "; + break; + } + case 329: { + s += " LAST "; + break; + } + case 330: { + s += " LAYOUT "; + break; + } + case 331: { + s += " LEADING "; + break; + } + case 332: { + s += " LEFT "; + break; + } + case 333: { + s += " LEFT ARRAY JOIN "; + break; + } + case 334: { + s += " length "; + break; + } + case 335: { + s += " LIFETIME "; + break; + } + case 336: { + s += " LIKE "; + break; + } + case 337: { + s += " LIMIT "; + break; + } + case 338: { + s += " LIVE "; + break; + } + case 339: { + s += " LOCAL "; + break; + } + case 340: { + s += " LowCardinality "; + break; + } + case 341: { + s += " LTRIM "; + break; + } + case 342: { + s += " M "; + break; + } + case 343: { + s += " MATERIALIZED "; + break; + } + case 344: { + s += " MATERIALIZE INDEX "; + break; + } + case 345: { + s += " MATERIALIZE TTL "; + break; + } + case 346: { + s += " max "; + break; + } + case 347: { + s += " maxIntersections "; + break; + } + case 348: { + s += " maxIntersectionsPosition "; + break; + } + case 349: { + s += " Merge "; + break; + } + case 350: { + s += " MI "; + break; + } + case 351: { + s += " min "; + break; + } + case 352: { + s += " MINUTE "; + break; + } + case 353: { + s += " MM "; + break; + } + case 354: { + s += " MODIFY "; + break; + } + case 355: { + s += " MODIFY COLUMN "; + break; + } + case 356: { + s += " MODIFY ORDER BY "; + break; + } + case 357: { + s += " MODIFY QUERY "; + break; + } + case 358: { + s += " MODIFY SETTING "; + break; + } + case 359: { + s += " MODIFY TTL "; + break; + } + case 360: { + s += " MONTH "; + break; + } + case 361: { + s += " MOVE PART "; + break; + } + case 362: { + s += " MOVE PARTITION "; + break; + } + case 363: { + s += " movingXXX "; + break; + } + case 364: { + s += " N "; + break; + } + case 365: { + s += " NAME "; + break; + } + case 366: { + s += " Nested "; + break; + } + case 367: { + s += " NO DELAY "; + break; + } + case 368: { + s += " NONE "; + break; + } + case 369: { + s += " not "; + break; + } + case 370: { + s += " nothing "; + break; + } + case 371: { + s += " Nothing "; + break; + } + case 372: { + s += " Null "; + break; + } + case 373: { + s += " Nullable "; + break; + } + case 374: { + s += " NULLS "; + break; + } + case 375: { + s += " OFFSET "; + break; + } + case 376: { + s += " ON "; + break; + } + case 377: { + s += " ONLY "; + break; + } + case 378: { + s += " OPTIMIZE TABLE "; + break; + } + case 379: { + s += " ORDER BY "; + break; + } + case 380: { + s += " OR REPLACE "; + break; + } + case 381: { + s += " OUTER "; + break; + } + case 382: { + s += " PARTITION "; + break; + } + case 383: { + s += " PARTITION BY "; + break; + } + case 384: { + s += " path "; + break; + } + case 385: { + s += " pathFull "; + break; + } + case 386: { + s += " POPULATE "; + break; + } + case 387: { + s += " PREWHERE "; + break; + } + case 388: { + s += " PRIMARY KEY "; + break; + } + case 389: { + s += " protocol "; + break; + } + case 390: { + s += " Q "; + break; + } + case 391: { + s += " QQ "; + break; + } + case 392: { + s += " QUARTER "; + break; + } + case 393: { + s += " queryString "; + break; + } + case 394: { + s += " queryStringAndFragment "; + break; + } + case 395: { + s += " range "; + break; + } + case 396: { + s += " REFRESH "; + break; + } + case 397: { + s += " RENAME COLUMN "; + break; + } + case 398: { + s += " RENAME TABLE "; + break; + } + case 399: { + s += " REPLACE PARTITION "; + break; + } + case 400: { + s += " Resample "; + break; + } + case 401: { + s += " RESUME "; + break; + } + case 402: { + s += " retention "; + break; + } + case 403: { + s += " RIGHT "; + break; + } + case 404: { + s += " ROLLUP "; + break; + } + case 405: { + s += " RTRIM "; + break; + } + case 406: { + s += " S "; + break; + } + case 407: { + s += " SAMPLE "; + break; + } + case 408: { + s += " SAMPLE BY "; + break; + } + case 409: { + s += " SECOND "; + break; + } + case 410: { + s += " SELECT "; + break; + } + case 411: { + s += " SEMI "; + break; + } + case 412: { + s += " sequenceCount "; + break; + } + case 413: { + s += " sequenceMatch "; + break; + } + case 414: { + s += " SET "; + break; + } + case 415: { + s += " SETTINGS "; + break; + } + case 416: { + s += " SHOW "; + break; + } + case 417: { + s += " SHOW PROCESSLIST "; + break; + } + case 418: { + s += " simpleLinearRegression "; + break; + } + case 419: { + s += " skewPop "; + break; + } + case 420: { + s += " skewSamp "; + break; + } + case 421: { + s += " SOURCE "; + break; + } + case 422: { + s += " SQL_TSI_DAY "; + break; + } + case 423: { + s += " SQL_TSI_HOUR "; + break; + } + case 424: { + s += " SQL_TSI_MINUTE "; + break; + } + case 425: { + s += " SQL_TSI_MONTH "; + break; + } + case 426: { + s += " SQL_TSI_QUARTER "; + break; + } + case 427: { + s += " SQL_TSI_SECOND "; + break; + } + case 428: { + s += " SQL_TSI_WEEK "; + break; + } + case 429: { + s += " SQL_TSI_YEAR "; + break; + } + case 430: { + s += " SS "; + break; + } + case 431: { + s += " State "; + break; + } + case 432: { + s += " stddevPop "; + break; + } + case 433: { + s += " stddevPopStable "; + break; + } + case 434: { + s += " stddevSamp "; + break; + } + case 435: { + s += " stddevSampStable "; + break; + } + case 436: { + s += " STEP "; + break; + } + case 437: { + s += " stochasticLinearRegression "; + break; + } + case 438: { + s += " stochasticLogisticRegression "; + break; + } + case 439: { + s += " String "; + break; + } + case 440: { + s += " SUBSTRING "; + break; + } + case 441: { + s += " sum "; + break; + } + case 442: { + s += " sumKahan "; + break; + } + case 443: { + s += " sumMap "; + break; + } + case 444: { + s += " sumMapFiltered "; + break; + } + case 445: { + s += " sumMapFilteredWithOverflow "; + break; + } + case 446: { + s += " sumMapWithOverflow "; + break; + } + case 447: { + s += " sumWithOverflow "; + break; + } + case 448: { + s += " SUSPEND "; + break; + } + case 449: { + s += " TABLE "; + break; + } + case 450: { + s += " TABLES "; + break; + } + case 451: { + s += " TEMPORARY "; + break; + } + case 452: { + s += " TIMESTAMP "; + break; + } + case 453: { + s += " TIMESTAMP_ADD "; + break; + } + case 454: { + s += " TIMESTAMPADD "; + break; + } + case 455: { + s += " TIMESTAMP_DIFF "; + break; + } + case 456: { + s += " TIMESTAMPDIFF "; + break; + } + case 457: { + s += " TIMESTAMP_SUB "; + break; + } + case 458: { + s += " TIMESTAMPSUB "; + break; + } + case 459: { + s += " TO "; + break; + } + case 460: { + s += " TO DISK "; + break; + } + case 461: { + s += " TOP "; + break; + } + case 462: { + s += " topK "; + break; + } + case 463: { + s += " topKWeighted "; + break; + } + case 464: { + s += " topLevelDomain "; + break; + } + case 465: { + s += " TO TABLE "; + break; + } + case 466: { + s += " TOTALS "; + break; + } + case 467: { + s += " TO VOLUME "; + break; + } + case 468: { + s += " TRAILING "; + break; + } + case 469: { + s += " TRIM "; + break; + } + case 470: { + s += " TRUNCATE "; + break; + } + case 471: { + s += " TTL "; + break; + } + case 472: { + s += " Tuple "; + break; + } + case 473: { + s += " TYPE "; + break; + } + case 474: { + s += " UInt16 "; + break; + } + case 475: { + s += " UInt32 "; + break; + } + case 476: { + s += " UInt64 "; + break; + } + case 477: { + s += " UInt8 "; + break; + } + case 478: { + s += " uniq "; + break; + } + case 479: { + s += " uniqCombined "; + break; + } + case 480: { + s += " uniqCombined64 "; + break; + } + case 481: { + s += " uniqExact "; + break; + } + case 482: { + s += " uniqHLL12 "; + break; + } + case 483: { + s += " uniqUpTo "; + break; + } + case 484: { + s += " UPDATE "; + break; + } + case 485: { + s += " URLHierarchy "; + break; + } + case 486: { + s += " URLPathHierarchy "; + break; + } + case 487: { + s += " USE "; + break; + } + case 488: { + s += " USING "; + break; + } + case 489: { + s += " UUID "; + break; + } + case 490: { + s += " VALUES "; + break; + } + case 491: { + s += " varPop "; + break; + } + case 492: { + s += " varPopStable "; + break; + } + case 493: { + s += " varSamp "; + break; + } + case 494: { + s += " varSampStable "; + break; + } + case 495: { + s += " VIEW "; + break; + } + case 496: { + s += " WATCH "; + break; + } + case 497: { + s += " WEEK "; + break; + } + case 498: { + s += " WHERE "; + break; + } + case 499: { + s += " windowFunnel "; + break; + } + case 500: { + s += " WITH "; + break; + } + case 501: { + s += " WITH FILL "; + break; + } + case 502: { + s += " WITH TIES "; + break; + } + case 503: { + s += " WK "; + break; + } + case 504: { + s += " WW "; + break; + } + case 505: { + s += " YEAR "; + break; + } + case 506: { + s += " YY "; + break; + } + case 507: { + s += " YYYY "; + break; + } + default: break; + } +} diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.proto b/src/Parsers/fuzzers/codegen_fuzzer/out.proto new file mode 100644 index 00000000000..6c8cefce9a3 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/out.proto @@ -0,0 +1,519 @@ +syntax = "proto3"; + +message Word { + enum Value { + value_0 = 0; + value_1 = 1; + value_2 = 2; + value_3 = 3; + value_4 = 4; + value_5 = 5; + value_6 = 6; + value_7 = 7; + value_8 = 8; + value_9 = 9; + value_10 = 10; + value_11 = 11; + value_12 = 12; + value_13 = 13; + value_14 = 14; + value_15 = 15; + value_16 = 16; + value_17 = 17; + value_18 = 18; + value_19 = 19; + value_20 = 20; + value_21 = 21; + value_22 = 22; + value_23 = 23; + value_24 = 24; + value_25 = 25; + value_26 = 26; + value_27 = 27; + value_28 = 28; + value_29 = 29; + value_30 = 30; + value_31 = 31; + value_32 = 32; + value_33 = 33; + value_34 = 34; + value_35 = 35; + value_36 = 36; + value_37 = 37; + value_38 = 38; + value_39 = 39; + value_40 = 40; + value_41 = 41; + value_42 = 42; + value_43 = 43; + value_44 = 44; + value_45 = 45; + value_46 = 46; + value_47 = 47; + value_48 = 48; + value_49 = 49; + value_50 = 50; + value_51 = 51; + value_52 = 52; + value_53 = 53; + value_54 = 54; + value_55 = 55; + value_56 = 56; + value_57 = 57; + value_58 = 58; + value_59 = 59; + value_60 = 60; + value_61 = 61; + value_62 = 62; + value_63 = 63; + value_64 = 64; + value_65 = 65; + value_66 = 66; + value_67 = 67; + value_68 = 68; + value_69 = 69; + value_70 = 70; + value_71 = 71; + value_72 = 72; + value_73 = 73; + value_74 = 74; + value_75 = 75; + value_76 = 76; + value_77 = 77; + value_78 = 78; + value_79 = 79; + value_80 = 80; + value_81 = 81; + value_82 = 82; + value_83 = 83; + value_84 = 84; + value_85 = 85; + value_86 = 86; + value_87 = 87; + value_88 = 88; + value_89 = 89; + value_90 = 90; + value_91 = 91; + value_92 = 92; + value_93 = 93; + value_94 = 94; + value_95 = 95; + value_96 = 96; + value_97 = 97; + value_98 = 98; + value_99 = 99; + value_100 = 100; + value_101 = 101; + value_102 = 102; + value_103 = 103; + value_104 = 104; + value_105 = 105; + value_106 = 106; + value_107 = 107; + value_108 = 108; + value_109 = 109; + value_110 = 110; + value_111 = 111; + value_112 = 112; + value_113 = 113; + value_114 = 114; + value_115 = 115; + value_116 = 116; + value_117 = 117; + value_118 = 118; + value_119 = 119; + value_120 = 120; + value_121 = 121; + value_122 = 122; + value_123 = 123; + value_124 = 124; + value_125 = 125; + value_126 = 126; + value_127 = 127; + value_128 = 128; + value_129 = 129; + value_130 = 130; + value_131 = 131; + value_132 = 132; + value_133 = 133; + value_134 = 134; + value_135 = 135; + value_136 = 136; + value_137 = 137; + value_138 = 138; + value_139 = 139; + value_140 = 140; + value_141 = 141; + value_142 = 142; + value_143 = 143; + value_144 = 144; + value_145 = 145; + value_146 = 146; + value_147 = 147; + value_148 = 148; + value_149 = 149; + value_150 = 150; + value_151 = 151; + value_152 = 152; + value_153 = 153; + value_154 = 154; + value_155 = 155; + value_156 = 156; + value_157 = 157; + value_158 = 158; + value_159 = 159; + value_160 = 160; + value_161 = 161; + value_162 = 162; + value_163 = 163; + value_164 = 164; + value_165 = 165; + value_166 = 166; + value_167 = 167; + value_168 = 168; + value_169 = 169; + value_170 = 170; + value_171 = 171; + value_172 = 172; + value_173 = 173; + value_174 = 174; + value_175 = 175; + value_176 = 176; + value_177 = 177; + value_178 = 178; + value_179 = 179; + value_180 = 180; + value_181 = 181; + value_182 = 182; + value_183 = 183; + value_184 = 184; + value_185 = 185; + value_186 = 186; + value_187 = 187; + value_188 = 188; + value_189 = 189; + value_190 = 190; + value_191 = 191; + value_192 = 192; + value_193 = 193; + value_194 = 194; + value_195 = 195; + value_196 = 196; + value_197 = 197; + value_198 = 198; + value_199 = 199; + value_200 = 200; + value_201 = 201; + value_202 = 202; + value_203 = 203; + value_204 = 204; + value_205 = 205; + value_206 = 206; + value_207 = 207; + value_208 = 208; + value_209 = 209; + value_210 = 210; + value_211 = 211; + value_212 = 212; + value_213 = 213; + value_214 = 214; + value_215 = 215; + value_216 = 216; + value_217 = 217; + value_218 = 218; + value_219 = 219; + value_220 = 220; + value_221 = 221; + value_222 = 222; + value_223 = 223; + value_224 = 224; + value_225 = 225; + value_226 = 226; + value_227 = 227; + value_228 = 228; + value_229 = 229; + value_230 = 230; + value_231 = 231; + value_232 = 232; + value_233 = 233; + value_234 = 234; + value_235 = 235; + value_236 = 236; + value_237 = 237; + value_238 = 238; + value_239 = 239; + value_240 = 240; + value_241 = 241; + value_242 = 242; + value_243 = 243; + value_244 = 244; + value_245 = 245; + value_246 = 246; + value_247 = 247; + value_248 = 248; + value_249 = 249; + value_250 = 250; + value_251 = 251; + value_252 = 252; + value_253 = 253; + value_254 = 254; + value_255 = 255; + value_256 = 256; + value_257 = 257; + value_258 = 258; + value_259 = 259; + value_260 = 260; + value_261 = 261; + value_262 = 262; + value_263 = 263; + value_264 = 264; + value_265 = 265; + value_266 = 266; + value_267 = 267; + value_268 = 268; + value_269 = 269; + value_270 = 270; + value_271 = 271; + value_272 = 272; + value_273 = 273; + value_274 = 274; + value_275 = 275; + value_276 = 276; + value_277 = 277; + value_278 = 278; + value_279 = 279; + value_280 = 280; + value_281 = 281; + value_282 = 282; + value_283 = 283; + value_284 = 284; + value_285 = 285; + value_286 = 286; + value_287 = 287; + value_288 = 288; + value_289 = 289; + value_290 = 290; + value_291 = 291; + value_292 = 292; + value_293 = 293; + value_294 = 294; + value_295 = 295; + value_296 = 296; + value_297 = 297; + value_298 = 298; + value_299 = 299; + value_300 = 300; + value_301 = 301; + value_302 = 302; + value_303 = 303; + value_304 = 304; + value_305 = 305; + value_306 = 306; + value_307 = 307; + value_308 = 308; + value_309 = 309; + value_310 = 310; + value_311 = 311; + value_312 = 312; + value_313 = 313; + value_314 = 314; + value_315 = 315; + value_316 = 316; + value_317 = 317; + value_318 = 318; + value_319 = 319; + value_320 = 320; + value_321 = 321; + value_322 = 322; + value_323 = 323; + value_324 = 324; + value_325 = 325; + value_326 = 326; + value_327 = 327; + value_328 = 328; + value_329 = 329; + value_330 = 330; + value_331 = 331; + value_332 = 332; + value_333 = 333; + value_334 = 334; + value_335 = 335; + value_336 = 336; + value_337 = 337; + value_338 = 338; + value_339 = 339; + value_340 = 340; + value_341 = 341; + value_342 = 342; + value_343 = 343; + value_344 = 344; + value_345 = 345; + value_346 = 346; + value_347 = 347; + value_348 = 348; + value_349 = 349; + value_350 = 350; + value_351 = 351; + value_352 = 352; + value_353 = 353; + value_354 = 354; + value_355 = 355; + value_356 = 356; + value_357 = 357; + value_358 = 358; + value_359 = 359; + value_360 = 360; + value_361 = 361; + value_362 = 362; + value_363 = 363; + value_364 = 364; + value_365 = 365; + value_366 = 366; + value_367 = 367; + value_368 = 368; + value_369 = 369; + value_370 = 370; + value_371 = 371; + value_372 = 372; + value_373 = 373; + value_374 = 374; + value_375 = 375; + value_376 = 376; + value_377 = 377; + value_378 = 378; + value_379 = 379; + value_380 = 380; + value_381 = 381; + value_382 = 382; + value_383 = 383; + value_384 = 384; + value_385 = 385; + value_386 = 386; + value_387 = 387; + value_388 = 388; + value_389 = 389; + value_390 = 390; + value_391 = 391; + value_392 = 392; + value_393 = 393; + value_394 = 394; + value_395 = 395; + value_396 = 396; + value_397 = 397; + value_398 = 398; + value_399 = 399; + value_400 = 400; + value_401 = 401; + value_402 = 402; + value_403 = 403; + value_404 = 404; + value_405 = 405; + value_406 = 406; + value_407 = 407; + value_408 = 408; + value_409 = 409; + value_410 = 410; + value_411 = 411; + value_412 = 412; + value_413 = 413; + value_414 = 414; + value_415 = 415; + value_416 = 416; + value_417 = 417; + value_418 = 418; + value_419 = 419; + value_420 = 420; + value_421 = 421; + value_422 = 422; + value_423 = 423; + value_424 = 424; + value_425 = 425; + value_426 = 426; + value_427 = 427; + value_428 = 428; + value_429 = 429; + value_430 = 430; + value_431 = 431; + value_432 = 432; + value_433 = 433; + value_434 = 434; + value_435 = 435; + value_436 = 436; + value_437 = 437; + value_438 = 438; + value_439 = 439; + value_440 = 440; + value_441 = 441; + value_442 = 442; + value_443 = 443; + value_444 = 444; + value_445 = 445; + value_446 = 446; + value_447 = 447; + value_448 = 448; + value_449 = 449; + value_450 = 450; + value_451 = 451; + value_452 = 452; + value_453 = 453; + value_454 = 454; + value_455 = 455; + value_456 = 456; + value_457 = 457; + value_458 = 458; + value_459 = 459; + value_460 = 460; + value_461 = 461; + value_462 = 462; + value_463 = 463; + value_464 = 464; + value_465 = 465; + value_466 = 466; + value_467 = 467; + value_468 = 468; + value_469 = 469; + value_470 = 470; + value_471 = 471; + value_472 = 472; + value_473 = 473; + value_474 = 474; + value_475 = 475; + value_476 = 476; + value_477 = 477; + value_478 = 478; + value_479 = 479; + value_480 = 480; + value_481 = 481; + value_482 = 482; + value_483 = 483; + value_484 = 484; + value_485 = 485; + value_486 = 486; + value_487 = 487; + value_488 = 488; + value_489 = 489; + value_490 = 490; + value_491 = 491; + value_492 = 492; + value_493 = 493; + value_494 = 494; + value_495 = 495; + value_496 = 496; + value_497 = 497; + value_498 = 498; + value_499 = 499; + value_500 = 500; + value_501 = 501; + value_502 = 502; + value_503 = 503; + value_504 = 504; + value_505 = 505; + value_506 = 506; + value_507 = 507; + } + Value value = 1; + Sentence inner = 2; +} +message Sentence { + repeated Word words = 1; +} \ No newline at end of file diff --git a/src/Parsers/fuzzers/codegen_fuzzer/update.sh b/src/Parsers/fuzzers/codegen_fuzzer/update.sh new file mode 100755 index 00000000000..0982d6d0686 --- /dev/null +++ b/src/Parsers/fuzzers/codegen_fuzzer/update.sh @@ -0,0 +1,30 @@ +#!/bin/bash + + +_main() { + local dict_filename="${1}" + if [[ $# -ne 1 ]]; + then + echo "Usage: $0 "; + exit 1; + fi + + if [[ ! -f $dict_filename ]]; + then + echo "File $dict_filename doesn't exist"; + exit 1 + fi + + cat clickhouse-template.g > clickhouse.g + + while read line; + do + [[ -z "$line" ]] && continue + echo $line | sed -e 's/"\(.*\)"/" \1 ";/g' + done < $dict_filename >> clickhouse.g +} + +_main "$@" + +# Sample run: ./update.sh ../../../../tests/fuzz/ast.dict +# then run `python ./gen.py clickhouse.g out.cpp out.proto` to generate new files with tokens. Rebuild fuzzer From 9ed33612980c59ac38ea63182407071148befd39 Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Tue, 12 Oct 2021 03:53:43 +0300 Subject: [PATCH 474/950] removing code generated files, since protobuf generation is now done in CMake --- src/Parsers/fuzzers/codegen_fuzzer/out.cpp | 2189 ------------------ src/Parsers/fuzzers/codegen_fuzzer/out.proto | 519 ----- 2 files changed, 2708 deletions(-) delete mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.cpp delete mode 100644 src/Parsers/fuzzers/codegen_fuzzer/out.proto diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.cpp b/src/Parsers/fuzzers/codegen_fuzzer/out.cpp deleted file mode 100644 index effe6e7821b..00000000000 --- a/src/Parsers/fuzzers/codegen_fuzzer/out.cpp +++ /dev/null @@ -1,2189 +0,0 @@ -#include -#include -#include - -#include - -#include "out.pb.h" - -void GenerateWord(const Word&, std::string&, int); - -void GenerateSentence(const Sentence& stc, std::string &s, int depth) { - for (int i = 0; i < stc.words_size(); i++ ) { - GenerateWord(stc.words(i), s, ++depth); - } -} -void GenerateWord(const Word& word, std::string &s, int depth) { - if (depth > 5) return; - - switch (word.value()) { - case 0: { - s += " "; - break; - } - case 1: { - s += " "; - break; - } - case 2: { - s += " "; - break; - } - case 3: { - s += ";"; - break; - } - case 4: { - s += "("; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ")"; - break; - } - case 5: { - s += "("; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ")"; - break; - } - case 6: { - s += "("; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ")"; - break; - } - case 7: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 8: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 9: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - break; - } - case 10: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += ", "; - if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); - break; - } - case 11: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += "]"; - break; - } - case 12: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += "]"; - break; - } - case 13: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += "]"; - break; - } - case 14: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ", "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += ", "; - if (word.inner().words_size() > 4) GenerateWord(word.inner().words(4), s, ++depth); - s += "]"; - break; - } - case 15: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += "("; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ")"; - break; - } - case 16: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += "("; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ")"; - break; - } - case 17: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += "("; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += ", "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += ", "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - s += ")"; - break; - } - case 18: { - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " as "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 19: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " WHERE "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 20: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " GROUP BY "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 21: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " SORT BY "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 22: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " LIMIT "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 23: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 24: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " ARRAY JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - break; - } - case 25: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += " ON "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - break; - } - case 26: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " FROM "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += " JOIN "; - if (word.inner().words_size() > 2) GenerateWord(word.inner().words(2), s, ++depth); - s += " USING "; - if (word.inner().words_size() > 3) GenerateWord(word.inner().words(3), s, ++depth); - break; - } - case 27: { - s += "SELECT "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " INTO OUTFILE "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 28: { - s += "WITH "; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += " AS "; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - break; - } - case 29: { - s += "{"; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ":"; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += "}"; - break; - } - case 30: { - s += "["; - if (word.inner().words_size() > 0) GenerateWord(word.inner().words(0), s, ++depth); - s += ","; - if (word.inner().words_size() > 1) GenerateWord(word.inner().words(1), s, ++depth); - s += "]"; - break; - } - case 31: { - s += "[]"; - break; - } - case 32: { - s += " x "; - break; - } - case 33: { - s += "x"; - break; - } - case 34: { - s += " `x` "; - break; - } - case 35: { - s += "`x`"; - break; - } - case 36: { - s += " \"value\" "; - break; - } - case 37: { - s += "\"value\""; - break; - } - case 38: { - s += " 0 "; - break; - } - case 39: { - s += "0"; - break; - } - case 40: { - s += "1"; - break; - } - case 41: { - s += "2"; - break; - } - case 42: { - s += "123123123123123123"; - break; - } - case 43: { - s += "182374019873401982734091873420923123123123123123"; - break; - } - case 44: { - s += "1e-1"; - break; - } - case 45: { - s += "1.1"; - break; - } - case 46: { - s += "\"\""; - break; - } - case 47: { - s += " '../../../../../../../../../etc/passwd' "; - break; - } - case 48: { - s += "/"; - break; - } - case 49: { - s += "="; - break; - } - case 50: { - s += "=="; - break; - } - case 51: { - s += "!="; - break; - } - case 52: { - s += "<>"; - break; - } - case 53: { - s += "<"; - break; - } - case 54: { - s += "<="; - break; - } - case 55: { - s += ">"; - break; - } - case 56: { - s += ">="; - break; - } - case 57: { - s += "<<"; - break; - } - case 58: { - s += "|<<"; - break; - } - case 59: { - s += "&"; - break; - } - case 60: { - s += "|"; - break; - } - case 61: { - s += "||"; - break; - } - case 62: { - s += "<|"; - break; - } - case 63: { - s += "|>"; - break; - } - case 64: { - s += "+"; - break; - } - case 65: { - s += "-"; - break; - } - case 66: { - s += "~"; - break; - } - case 67: { - s += "*"; - break; - } - case 68: { - s += "/"; - break; - } - case 69: { - s += "\\"; - break; - } - case 70: { - s += "%"; - break; - } - case 71: { - s += ""; - break; - } - case 72: { - s += "."; - break; - } - case 73: { - s += ","; - break; - } - case 74: { - s += ","; - break; - } - case 75: { - s += ","; - break; - } - case 76: { - s += ","; - break; - } - case 77: { - s += ","; - break; - } - case 78: { - s += ","; - break; - } - case 79: { - s += "("; - break; - } - case 80: { - s += ")"; - break; - } - case 81: { - s += "("; - break; - } - case 82: { - s += ")"; - break; - } - case 83: { - s += "("; - break; - } - case 84: { - s += ")"; - break; - } - case 85: { - s += "("; - break; - } - case 86: { - s += ")"; - break; - } - case 87: { - s += "("; - break; - } - case 88: { - s += ")"; - break; - } - case 89: { - s += "("; - break; - } - case 90: { - s += ")"; - break; - } - case 91: { - s += "?"; - break; - } - case 92: { - s += ":"; - break; - } - case 93: { - s += "@"; - break; - } - case 94: { - s += "@@"; - break; - } - case 95: { - s += "$"; - break; - } - case 96: { - s += "\""; - break; - } - case 97: { - s += "`"; - break; - } - case 98: { - s += "{"; - break; - } - case 99: { - s += "}"; - break; - } - case 100: { - s += "^"; - break; - } - case 101: { - s += "::"; - break; - } - case 102: { - s += "->"; - break; - } - case 103: { - s += "]"; - break; - } - case 104: { - s += "["; - break; - } - case 105: { - s += " ADD "; - break; - } - case 106: { - s += " ADD COLUMN "; - break; - } - case 107: { - s += " ADD CONSTRAINT "; - break; - } - case 108: { - s += " ADD INDEX "; - break; - } - case 109: { - s += " AFTER "; - break; - } - case 110: { - s += " AggregateFunction "; - break; - } - case 111: { - s += " aggThrow "; - break; - } - case 112: { - s += " ALL "; - break; - } - case 113: { - s += " ALTER LIVE VIEW "; - break; - } - case 114: { - s += " ALTER TABLE "; - break; - } - case 115: { - s += " and "; - break; - } - case 116: { - s += " ANTI "; - break; - } - case 117: { - s += " any "; - break; - } - case 118: { - s += " anyHeavy "; - break; - } - case 119: { - s += " anyLast "; - break; - } - case 120: { - s += " argMax "; - break; - } - case 121: { - s += " argMin "; - break; - } - case 122: { - s += " array "; - break; - } - case 123: { - s += " Array "; - break; - } - case 124: { - s += " arrayAll "; - break; - } - case 125: { - s += " arrayAUC "; - break; - } - case 126: { - s += " arrayCompact "; - break; - } - case 127: { - s += " arrayConcat "; - break; - } - case 128: { - s += " arrayCount "; - break; - } - case 129: { - s += " arrayCumSum "; - break; - } - case 130: { - s += " arrayCumSumNonNegative "; - break; - } - case 131: { - s += " arrayDifference "; - break; - } - case 132: { - s += " arrayDistinct "; - break; - } - case 133: { - s += " arrayElement "; - break; - } - case 134: { - s += " arrayEnumerate "; - break; - } - case 135: { - s += " arrayEnumerateDense "; - break; - } - case 136: { - s += " arrayEnumerateDenseRanked "; - break; - } - case 137: { - s += " arrayEnumerateUniq "; - break; - } - case 138: { - s += " arrayEnumerateUniqRanked "; - break; - } - case 139: { - s += " arrayExists "; - break; - } - case 140: { - s += " arrayFill "; - break; - } - case 141: { - s += " arrayFilter "; - break; - } - case 142: { - s += " arrayFirst "; - break; - } - case 143: { - s += " arrayFirstIndex "; - break; - } - case 144: { - s += " arrayFlatten "; - break; - } - case 145: { - s += " arrayIntersect "; - break; - } - case 146: { - s += " arrayJoin "; - break; - } - case 147: { - s += " ARRAY JOIN "; - break; - } - case 148: { - s += " arrayMap "; - break; - } - case 149: { - s += " arrayPopBack "; - break; - } - case 150: { - s += " arrayPopFront "; - break; - } - case 151: { - s += " arrayPushBack "; - break; - } - case 152: { - s += " arrayPushFront "; - break; - } - case 153: { - s += " arrayReduce "; - break; - } - case 154: { - s += " arrayReduceInRanges "; - break; - } - case 155: { - s += " arrayResize "; - break; - } - case 156: { - s += " arrayReverse "; - break; - } - case 157: { - s += " arrayReverseFill "; - break; - } - case 158: { - s += " arrayReverseSort "; - break; - } - case 159: { - s += " arrayReverseSplit "; - break; - } - case 160: { - s += " arraySlice "; - break; - } - case 161: { - s += " arraySort "; - break; - } - case 162: { - s += " arraySplit "; - break; - } - case 163: { - s += " arraySum "; - break; - } - case 164: { - s += " arrayUniq "; - break; - } - case 165: { - s += " arrayWithConstant "; - break; - } - case 166: { - s += " arrayZip "; - break; - } - case 167: { - s += " AS "; - break; - } - case 168: { - s += " ASC "; - break; - } - case 169: { - s += " ASCENDING "; - break; - } - case 170: { - s += " ASOF "; - break; - } - case 171: { - s += " AST "; - break; - } - case 172: { - s += " ATTACH "; - break; - } - case 173: { - s += " ATTACH PART "; - break; - } - case 174: { - s += " ATTACH PARTITION "; - break; - } - case 175: { - s += " avg "; - break; - } - case 176: { - s += " avgWeighted "; - break; - } - case 177: { - s += " basename "; - break; - } - case 178: { - s += " BETWEEN "; - break; - } - case 179: { - s += " BOTH "; - break; - } - case 180: { - s += " boundingRatio "; - break; - } - case 181: { - s += " BY "; - break; - } - case 182: { - s += " CAST "; - break; - } - case 183: { - s += " categoricalInformationValue "; - break; - } - case 184: { - s += " CHECK "; - break; - } - case 185: { - s += " CHECK TABLE "; - break; - } - case 186: { - s += " CLEAR COLUMN "; - break; - } - case 187: { - s += " CLEAR INDEX "; - break; - } - case 188: { - s += " COLLATE "; - break; - } - case 189: { - s += " COLUMNS "; - break; - } - case 190: { - s += " COMMENT COLUMN "; - break; - } - case 191: { - s += " CONSTRAINT "; - break; - } - case 192: { - s += " corr "; - break; - } - case 193: { - s += " corrStable "; - break; - } - case 194: { - s += " count "; - break; - } - case 195: { - s += " countEqual "; - break; - } - case 196: { - s += " covarPop "; - break; - } - case 197: { - s += " covarPopStable "; - break; - } - case 198: { - s += " covarSamp "; - break; - } - case 199: { - s += " covarSampStable "; - break; - } - case 200: { - s += " CREATE "; - break; - } - case 201: { - s += " CROSS "; - break; - } - case 202: { - s += " CUBE "; - break; - } - case 203: { - s += " cutFragment "; - break; - } - case 204: { - s += " cutQueryString "; - break; - } - case 205: { - s += " cutQueryStringAndFragment "; - break; - } - case 206: { - s += " cutToFirstSignificantSubdomain "; - break; - } - case 207: { - s += " cutURLParameter "; - break; - } - case 208: { - s += " cutWWW "; - break; - } - case 209: { - s += " D "; - break; - } - case 210: { - s += " DATABASE "; - break; - } - case 211: { - s += " DATABASES "; - break; - } - case 212: { - s += " Date "; - break; - } - case 213: { - s += " DATE "; - break; - } - case 214: { - s += " DATE_ADD "; - break; - } - case 215: { - s += " DATEADD "; - break; - } - case 216: { - s += " DATE_DIFF "; - break; - } - case 217: { - s += " DATEDIFF "; - break; - } - case 218: { - s += " DATE_SUB "; - break; - } - case 219: { - s += " DATESUB "; - break; - } - case 220: { - s += " DateTime "; - break; - } - case 221: { - s += " DateTime64 "; - break; - } - case 222: { - s += " DAY "; - break; - } - case 223: { - s += " DD "; - break; - } - case 224: { - s += " Decimal "; - break; - } - case 225: { - s += " Decimal128 "; - break; - } - case 226: { - s += " Decimal32 "; - break; - } - case 227: { - s += " Decimal64 "; - break; - } - case 228: { - s += " decodeURLComponent "; - break; - } - case 229: { - s += " DEDUPLICATE "; - break; - } - case 230: { - s += " DELETE "; - break; - } - case 231: { - s += " DELETE WHERE "; - break; - } - case 232: { - s += " DESC "; - break; - } - case 233: { - s += " DESCENDING "; - break; - } - case 234: { - s += " DESCRIBE "; - break; - } - case 235: { - s += " DETACH "; - break; - } - case 236: { - s += " DETACH PARTITION "; - break; - } - case 237: { - s += " DICTIONARIES "; - break; - } - case 238: { - s += " DICTIONARY "; - break; - } - case 239: { - s += " DISTINCT "; - break; - } - case 240: { - s += " domain "; - break; - } - case 241: { - s += " domainWithoutWWW "; - break; - } - case 242: { - s += " DROP "; - break; - } - case 243: { - s += " DROP COLUMN "; - break; - } - case 244: { - s += " DROP CONSTRAINT "; - break; - } - case 245: { - s += " DROP DETACHED PART "; - break; - } - case 246: { - s += " DROP DETACHED PARTITION "; - break; - } - case 247: { - s += " DROP INDEX "; - break; - } - case 248: { - s += " DROP PARTITION "; - break; - } - case 249: { - s += " emptyArrayToSingle "; - break; - } - case 250: { - s += " ENGINE "; - break; - } - case 251: { - s += " entropy "; - break; - } - case 252: { - s += " Enum "; - break; - } - case 253: { - s += " Enum16 "; - break; - } - case 254: { - s += " Enum8 "; - break; - } - case 255: { - s += " EVENTS "; - break; - } - case 256: { - s += " EXCHANGE TABLES "; - break; - } - case 257: { - s += " EXISTS "; - break; - } - case 258: { - s += " EXTRACT "; - break; - } - case 259: { - s += " extractURLParameter "; - break; - } - case 260: { - s += " extractURLParameterNames "; - break; - } - case 261: { - s += " extractURLParameters "; - break; - } - case 262: { - s += " FETCH PARTITION "; - break; - } - case 263: { - s += " FETCH PART "; - break; - } - case 264: { - s += " FINAL "; - break; - } - case 265: { - s += " FIRST "; - break; - } - case 266: { - s += " firstSignificantSubdomain "; - break; - } - case 267: { - s += " FixedString "; - break; - } - case 268: { - s += " Float32 "; - break; - } - case 269: { - s += " Float64 "; - break; - } - case 270: { - s += " FOR "; - break; - } - case 271: { - s += " ForEach "; - break; - } - case 272: { - s += " FORMAT "; - break; - } - case 273: { - s += " fragment "; - break; - } - case 274: { - s += " FREEZE "; - break; - } - case 275: { - s += " FROM "; - break; - } - case 276: { - s += " FULL "; - break; - } - case 277: { - s += " FUNCTION "; - break; - } - case 278: { - s += " __getScalar "; - break; - } - case 279: { - s += " GLOBAL "; - break; - } - case 280: { - s += " GRANULARITY "; - break; - } - case 281: { - s += " groupArray "; - break; - } - case 282: { - s += " groupArrayInsertAt "; - break; - } - case 283: { - s += " groupArrayMovingAvg "; - break; - } - case 284: { - s += " groupArrayMovingSum "; - break; - } - case 285: { - s += " groupArraySample "; - break; - } - case 286: { - s += " groupBitAnd "; - break; - } - case 287: { - s += " groupBitmap "; - break; - } - case 288: { - s += " groupBitmapAnd "; - break; - } - case 289: { - s += " groupBitmapOr "; - break; - } - case 290: { - s += " groupBitmapXor "; - break; - } - case 291: { - s += " groupBitOr "; - break; - } - case 292: { - s += " groupBitXor "; - break; - } - case 293: { - s += " GROUP BY "; - break; - } - case 294: { - s += " groupUniqArray "; - break; - } - case 295: { - s += " has "; - break; - } - case 296: { - s += " hasAll "; - break; - } - case 297: { - s += " hasAny "; - break; - } - case 298: { - s += " HAVING "; - break; - } - case 299: { - s += " HH "; - break; - } - case 300: { - s += " histogram "; - break; - } - case 301: { - s += " HOUR "; - break; - } - case 302: { - s += " ID "; - break; - } - case 303: { - s += " if "; - break; - } - case 304: { - s += " IF EXISTS "; - break; - } - case 305: { - s += " IF NOT EXISTS "; - break; - } - case 306: { - s += " IN "; - break; - } - case 307: { - s += " INDEX "; - break; - } - case 308: { - s += " indexOf "; - break; - } - case 309: { - s += " INNER "; - break; - } - case 310: { - s += " IN PARTITION "; - break; - } - case 311: { - s += " INSERT INTO "; - break; - } - case 312: { - s += " Int16 "; - break; - } - case 313: { - s += " Int32 "; - break; - } - case 314: { - s += " Int64 "; - break; - } - case 315: { - s += " Int8 "; - break; - } - case 316: { - s += " INTERVAL "; - break; - } - case 317: { - s += " IntervalDay "; - break; - } - case 318: { - s += " IntervalHour "; - break; - } - case 319: { - s += " IntervalMinute "; - break; - } - case 320: { - s += " IntervalMonth "; - break; - } - case 321: { - s += " IntervalQuarter "; - break; - } - case 322: { - s += " IntervalSecond "; - break; - } - case 323: { - s += " IntervalWeek "; - break; - } - case 324: { - s += " IntervalYear "; - break; - } - case 325: { - s += " INTO OUTFILE "; - break; - } - case 326: { - s += " JOIN "; - break; - } - case 327: { - s += " kurtPop "; - break; - } - case 328: { - s += " kurtSamp "; - break; - } - case 329: { - s += " LAST "; - break; - } - case 330: { - s += " LAYOUT "; - break; - } - case 331: { - s += " LEADING "; - break; - } - case 332: { - s += " LEFT "; - break; - } - case 333: { - s += " LEFT ARRAY JOIN "; - break; - } - case 334: { - s += " length "; - break; - } - case 335: { - s += " LIFETIME "; - break; - } - case 336: { - s += " LIKE "; - break; - } - case 337: { - s += " LIMIT "; - break; - } - case 338: { - s += " LIVE "; - break; - } - case 339: { - s += " LOCAL "; - break; - } - case 340: { - s += " LowCardinality "; - break; - } - case 341: { - s += " LTRIM "; - break; - } - case 342: { - s += " M "; - break; - } - case 343: { - s += " MATERIALIZED "; - break; - } - case 344: { - s += " MATERIALIZE INDEX "; - break; - } - case 345: { - s += " MATERIALIZE TTL "; - break; - } - case 346: { - s += " max "; - break; - } - case 347: { - s += " maxIntersections "; - break; - } - case 348: { - s += " maxIntersectionsPosition "; - break; - } - case 349: { - s += " Merge "; - break; - } - case 350: { - s += " MI "; - break; - } - case 351: { - s += " min "; - break; - } - case 352: { - s += " MINUTE "; - break; - } - case 353: { - s += " MM "; - break; - } - case 354: { - s += " MODIFY "; - break; - } - case 355: { - s += " MODIFY COLUMN "; - break; - } - case 356: { - s += " MODIFY ORDER BY "; - break; - } - case 357: { - s += " MODIFY QUERY "; - break; - } - case 358: { - s += " MODIFY SETTING "; - break; - } - case 359: { - s += " MODIFY TTL "; - break; - } - case 360: { - s += " MONTH "; - break; - } - case 361: { - s += " MOVE PART "; - break; - } - case 362: { - s += " MOVE PARTITION "; - break; - } - case 363: { - s += " movingXXX "; - break; - } - case 364: { - s += " N "; - break; - } - case 365: { - s += " NAME "; - break; - } - case 366: { - s += " Nested "; - break; - } - case 367: { - s += " NO DELAY "; - break; - } - case 368: { - s += " NONE "; - break; - } - case 369: { - s += " not "; - break; - } - case 370: { - s += " nothing "; - break; - } - case 371: { - s += " Nothing "; - break; - } - case 372: { - s += " Null "; - break; - } - case 373: { - s += " Nullable "; - break; - } - case 374: { - s += " NULLS "; - break; - } - case 375: { - s += " OFFSET "; - break; - } - case 376: { - s += " ON "; - break; - } - case 377: { - s += " ONLY "; - break; - } - case 378: { - s += " OPTIMIZE TABLE "; - break; - } - case 379: { - s += " ORDER BY "; - break; - } - case 380: { - s += " OR REPLACE "; - break; - } - case 381: { - s += " OUTER "; - break; - } - case 382: { - s += " PARTITION "; - break; - } - case 383: { - s += " PARTITION BY "; - break; - } - case 384: { - s += " path "; - break; - } - case 385: { - s += " pathFull "; - break; - } - case 386: { - s += " POPULATE "; - break; - } - case 387: { - s += " PREWHERE "; - break; - } - case 388: { - s += " PRIMARY KEY "; - break; - } - case 389: { - s += " protocol "; - break; - } - case 390: { - s += " Q "; - break; - } - case 391: { - s += " QQ "; - break; - } - case 392: { - s += " QUARTER "; - break; - } - case 393: { - s += " queryString "; - break; - } - case 394: { - s += " queryStringAndFragment "; - break; - } - case 395: { - s += " range "; - break; - } - case 396: { - s += " REFRESH "; - break; - } - case 397: { - s += " RENAME COLUMN "; - break; - } - case 398: { - s += " RENAME TABLE "; - break; - } - case 399: { - s += " REPLACE PARTITION "; - break; - } - case 400: { - s += " Resample "; - break; - } - case 401: { - s += " RESUME "; - break; - } - case 402: { - s += " retention "; - break; - } - case 403: { - s += " RIGHT "; - break; - } - case 404: { - s += " ROLLUP "; - break; - } - case 405: { - s += " RTRIM "; - break; - } - case 406: { - s += " S "; - break; - } - case 407: { - s += " SAMPLE "; - break; - } - case 408: { - s += " SAMPLE BY "; - break; - } - case 409: { - s += " SECOND "; - break; - } - case 410: { - s += " SELECT "; - break; - } - case 411: { - s += " SEMI "; - break; - } - case 412: { - s += " sequenceCount "; - break; - } - case 413: { - s += " sequenceMatch "; - break; - } - case 414: { - s += " SET "; - break; - } - case 415: { - s += " SETTINGS "; - break; - } - case 416: { - s += " SHOW "; - break; - } - case 417: { - s += " SHOW PROCESSLIST "; - break; - } - case 418: { - s += " simpleLinearRegression "; - break; - } - case 419: { - s += " skewPop "; - break; - } - case 420: { - s += " skewSamp "; - break; - } - case 421: { - s += " SOURCE "; - break; - } - case 422: { - s += " SQL_TSI_DAY "; - break; - } - case 423: { - s += " SQL_TSI_HOUR "; - break; - } - case 424: { - s += " SQL_TSI_MINUTE "; - break; - } - case 425: { - s += " SQL_TSI_MONTH "; - break; - } - case 426: { - s += " SQL_TSI_QUARTER "; - break; - } - case 427: { - s += " SQL_TSI_SECOND "; - break; - } - case 428: { - s += " SQL_TSI_WEEK "; - break; - } - case 429: { - s += " SQL_TSI_YEAR "; - break; - } - case 430: { - s += " SS "; - break; - } - case 431: { - s += " State "; - break; - } - case 432: { - s += " stddevPop "; - break; - } - case 433: { - s += " stddevPopStable "; - break; - } - case 434: { - s += " stddevSamp "; - break; - } - case 435: { - s += " stddevSampStable "; - break; - } - case 436: { - s += " STEP "; - break; - } - case 437: { - s += " stochasticLinearRegression "; - break; - } - case 438: { - s += " stochasticLogisticRegression "; - break; - } - case 439: { - s += " String "; - break; - } - case 440: { - s += " SUBSTRING "; - break; - } - case 441: { - s += " sum "; - break; - } - case 442: { - s += " sumKahan "; - break; - } - case 443: { - s += " sumMap "; - break; - } - case 444: { - s += " sumMapFiltered "; - break; - } - case 445: { - s += " sumMapFilteredWithOverflow "; - break; - } - case 446: { - s += " sumMapWithOverflow "; - break; - } - case 447: { - s += " sumWithOverflow "; - break; - } - case 448: { - s += " SUSPEND "; - break; - } - case 449: { - s += " TABLE "; - break; - } - case 450: { - s += " TABLES "; - break; - } - case 451: { - s += " TEMPORARY "; - break; - } - case 452: { - s += " TIMESTAMP "; - break; - } - case 453: { - s += " TIMESTAMP_ADD "; - break; - } - case 454: { - s += " TIMESTAMPADD "; - break; - } - case 455: { - s += " TIMESTAMP_DIFF "; - break; - } - case 456: { - s += " TIMESTAMPDIFF "; - break; - } - case 457: { - s += " TIMESTAMP_SUB "; - break; - } - case 458: { - s += " TIMESTAMPSUB "; - break; - } - case 459: { - s += " TO "; - break; - } - case 460: { - s += " TO DISK "; - break; - } - case 461: { - s += " TOP "; - break; - } - case 462: { - s += " topK "; - break; - } - case 463: { - s += " topKWeighted "; - break; - } - case 464: { - s += " topLevelDomain "; - break; - } - case 465: { - s += " TO TABLE "; - break; - } - case 466: { - s += " TOTALS "; - break; - } - case 467: { - s += " TO VOLUME "; - break; - } - case 468: { - s += " TRAILING "; - break; - } - case 469: { - s += " TRIM "; - break; - } - case 470: { - s += " TRUNCATE "; - break; - } - case 471: { - s += " TTL "; - break; - } - case 472: { - s += " Tuple "; - break; - } - case 473: { - s += " TYPE "; - break; - } - case 474: { - s += " UInt16 "; - break; - } - case 475: { - s += " UInt32 "; - break; - } - case 476: { - s += " UInt64 "; - break; - } - case 477: { - s += " UInt8 "; - break; - } - case 478: { - s += " uniq "; - break; - } - case 479: { - s += " uniqCombined "; - break; - } - case 480: { - s += " uniqCombined64 "; - break; - } - case 481: { - s += " uniqExact "; - break; - } - case 482: { - s += " uniqHLL12 "; - break; - } - case 483: { - s += " uniqUpTo "; - break; - } - case 484: { - s += " UPDATE "; - break; - } - case 485: { - s += " URLHierarchy "; - break; - } - case 486: { - s += " URLPathHierarchy "; - break; - } - case 487: { - s += " USE "; - break; - } - case 488: { - s += " USING "; - break; - } - case 489: { - s += " UUID "; - break; - } - case 490: { - s += " VALUES "; - break; - } - case 491: { - s += " varPop "; - break; - } - case 492: { - s += " varPopStable "; - break; - } - case 493: { - s += " varSamp "; - break; - } - case 494: { - s += " varSampStable "; - break; - } - case 495: { - s += " VIEW "; - break; - } - case 496: { - s += " WATCH "; - break; - } - case 497: { - s += " WEEK "; - break; - } - case 498: { - s += " WHERE "; - break; - } - case 499: { - s += " windowFunnel "; - break; - } - case 500: { - s += " WITH "; - break; - } - case 501: { - s += " WITH FILL "; - break; - } - case 502: { - s += " WITH TIES "; - break; - } - case 503: { - s += " WK "; - break; - } - case 504: { - s += " WW "; - break; - } - case 505: { - s += " YEAR "; - break; - } - case 506: { - s += " YY "; - break; - } - case 507: { - s += " YYYY "; - break; - } - default: break; - } -} diff --git a/src/Parsers/fuzzers/codegen_fuzzer/out.proto b/src/Parsers/fuzzers/codegen_fuzzer/out.proto deleted file mode 100644 index 6c8cefce9a3..00000000000 --- a/src/Parsers/fuzzers/codegen_fuzzer/out.proto +++ /dev/null @@ -1,519 +0,0 @@ -syntax = "proto3"; - -message Word { - enum Value { - value_0 = 0; - value_1 = 1; - value_2 = 2; - value_3 = 3; - value_4 = 4; - value_5 = 5; - value_6 = 6; - value_7 = 7; - value_8 = 8; - value_9 = 9; - value_10 = 10; - value_11 = 11; - value_12 = 12; - value_13 = 13; - value_14 = 14; - value_15 = 15; - value_16 = 16; - value_17 = 17; - value_18 = 18; - value_19 = 19; - value_20 = 20; - value_21 = 21; - value_22 = 22; - value_23 = 23; - value_24 = 24; - value_25 = 25; - value_26 = 26; - value_27 = 27; - value_28 = 28; - value_29 = 29; - value_30 = 30; - value_31 = 31; - value_32 = 32; - value_33 = 33; - value_34 = 34; - value_35 = 35; - value_36 = 36; - value_37 = 37; - value_38 = 38; - value_39 = 39; - value_40 = 40; - value_41 = 41; - value_42 = 42; - value_43 = 43; - value_44 = 44; - value_45 = 45; - value_46 = 46; - value_47 = 47; - value_48 = 48; - value_49 = 49; - value_50 = 50; - value_51 = 51; - value_52 = 52; - value_53 = 53; - value_54 = 54; - value_55 = 55; - value_56 = 56; - value_57 = 57; - value_58 = 58; - value_59 = 59; - value_60 = 60; - value_61 = 61; - value_62 = 62; - value_63 = 63; - value_64 = 64; - value_65 = 65; - value_66 = 66; - value_67 = 67; - value_68 = 68; - value_69 = 69; - value_70 = 70; - value_71 = 71; - value_72 = 72; - value_73 = 73; - value_74 = 74; - value_75 = 75; - value_76 = 76; - value_77 = 77; - value_78 = 78; - value_79 = 79; - value_80 = 80; - value_81 = 81; - value_82 = 82; - value_83 = 83; - value_84 = 84; - value_85 = 85; - value_86 = 86; - value_87 = 87; - value_88 = 88; - value_89 = 89; - value_90 = 90; - value_91 = 91; - value_92 = 92; - value_93 = 93; - value_94 = 94; - value_95 = 95; - value_96 = 96; - value_97 = 97; - value_98 = 98; - value_99 = 99; - value_100 = 100; - value_101 = 101; - value_102 = 102; - value_103 = 103; - value_104 = 104; - value_105 = 105; - value_106 = 106; - value_107 = 107; - value_108 = 108; - value_109 = 109; - value_110 = 110; - value_111 = 111; - value_112 = 112; - value_113 = 113; - value_114 = 114; - value_115 = 115; - value_116 = 116; - value_117 = 117; - value_118 = 118; - value_119 = 119; - value_120 = 120; - value_121 = 121; - value_122 = 122; - value_123 = 123; - value_124 = 124; - value_125 = 125; - value_126 = 126; - value_127 = 127; - value_128 = 128; - value_129 = 129; - value_130 = 130; - value_131 = 131; - value_132 = 132; - value_133 = 133; - value_134 = 134; - value_135 = 135; - value_136 = 136; - value_137 = 137; - value_138 = 138; - value_139 = 139; - value_140 = 140; - value_141 = 141; - value_142 = 142; - value_143 = 143; - value_144 = 144; - value_145 = 145; - value_146 = 146; - value_147 = 147; - value_148 = 148; - value_149 = 149; - value_150 = 150; - value_151 = 151; - value_152 = 152; - value_153 = 153; - value_154 = 154; - value_155 = 155; - value_156 = 156; - value_157 = 157; - value_158 = 158; - value_159 = 159; - value_160 = 160; - value_161 = 161; - value_162 = 162; - value_163 = 163; - value_164 = 164; - value_165 = 165; - value_166 = 166; - value_167 = 167; - value_168 = 168; - value_169 = 169; - value_170 = 170; - value_171 = 171; - value_172 = 172; - value_173 = 173; - value_174 = 174; - value_175 = 175; - value_176 = 176; - value_177 = 177; - value_178 = 178; - value_179 = 179; - value_180 = 180; - value_181 = 181; - value_182 = 182; - value_183 = 183; - value_184 = 184; - value_185 = 185; - value_186 = 186; - value_187 = 187; - value_188 = 188; - value_189 = 189; - value_190 = 190; - value_191 = 191; - value_192 = 192; - value_193 = 193; - value_194 = 194; - value_195 = 195; - value_196 = 196; - value_197 = 197; - value_198 = 198; - value_199 = 199; - value_200 = 200; - value_201 = 201; - value_202 = 202; - value_203 = 203; - value_204 = 204; - value_205 = 205; - value_206 = 206; - value_207 = 207; - value_208 = 208; - value_209 = 209; - value_210 = 210; - value_211 = 211; - value_212 = 212; - value_213 = 213; - value_214 = 214; - value_215 = 215; - value_216 = 216; - value_217 = 217; - value_218 = 218; - value_219 = 219; - value_220 = 220; - value_221 = 221; - value_222 = 222; - value_223 = 223; - value_224 = 224; - value_225 = 225; - value_226 = 226; - value_227 = 227; - value_228 = 228; - value_229 = 229; - value_230 = 230; - value_231 = 231; - value_232 = 232; - value_233 = 233; - value_234 = 234; - value_235 = 235; - value_236 = 236; - value_237 = 237; - value_238 = 238; - value_239 = 239; - value_240 = 240; - value_241 = 241; - value_242 = 242; - value_243 = 243; - value_244 = 244; - value_245 = 245; - value_246 = 246; - value_247 = 247; - value_248 = 248; - value_249 = 249; - value_250 = 250; - value_251 = 251; - value_252 = 252; - value_253 = 253; - value_254 = 254; - value_255 = 255; - value_256 = 256; - value_257 = 257; - value_258 = 258; - value_259 = 259; - value_260 = 260; - value_261 = 261; - value_262 = 262; - value_263 = 263; - value_264 = 264; - value_265 = 265; - value_266 = 266; - value_267 = 267; - value_268 = 268; - value_269 = 269; - value_270 = 270; - value_271 = 271; - value_272 = 272; - value_273 = 273; - value_274 = 274; - value_275 = 275; - value_276 = 276; - value_277 = 277; - value_278 = 278; - value_279 = 279; - value_280 = 280; - value_281 = 281; - value_282 = 282; - value_283 = 283; - value_284 = 284; - value_285 = 285; - value_286 = 286; - value_287 = 287; - value_288 = 288; - value_289 = 289; - value_290 = 290; - value_291 = 291; - value_292 = 292; - value_293 = 293; - value_294 = 294; - value_295 = 295; - value_296 = 296; - value_297 = 297; - value_298 = 298; - value_299 = 299; - value_300 = 300; - value_301 = 301; - value_302 = 302; - value_303 = 303; - value_304 = 304; - value_305 = 305; - value_306 = 306; - value_307 = 307; - value_308 = 308; - value_309 = 309; - value_310 = 310; - value_311 = 311; - value_312 = 312; - value_313 = 313; - value_314 = 314; - value_315 = 315; - value_316 = 316; - value_317 = 317; - value_318 = 318; - value_319 = 319; - value_320 = 320; - value_321 = 321; - value_322 = 322; - value_323 = 323; - value_324 = 324; - value_325 = 325; - value_326 = 326; - value_327 = 327; - value_328 = 328; - value_329 = 329; - value_330 = 330; - value_331 = 331; - value_332 = 332; - value_333 = 333; - value_334 = 334; - value_335 = 335; - value_336 = 336; - value_337 = 337; - value_338 = 338; - value_339 = 339; - value_340 = 340; - value_341 = 341; - value_342 = 342; - value_343 = 343; - value_344 = 344; - value_345 = 345; - value_346 = 346; - value_347 = 347; - value_348 = 348; - value_349 = 349; - value_350 = 350; - value_351 = 351; - value_352 = 352; - value_353 = 353; - value_354 = 354; - value_355 = 355; - value_356 = 356; - value_357 = 357; - value_358 = 358; - value_359 = 359; - value_360 = 360; - value_361 = 361; - value_362 = 362; - value_363 = 363; - value_364 = 364; - value_365 = 365; - value_366 = 366; - value_367 = 367; - value_368 = 368; - value_369 = 369; - value_370 = 370; - value_371 = 371; - value_372 = 372; - value_373 = 373; - value_374 = 374; - value_375 = 375; - value_376 = 376; - value_377 = 377; - value_378 = 378; - value_379 = 379; - value_380 = 380; - value_381 = 381; - value_382 = 382; - value_383 = 383; - value_384 = 384; - value_385 = 385; - value_386 = 386; - value_387 = 387; - value_388 = 388; - value_389 = 389; - value_390 = 390; - value_391 = 391; - value_392 = 392; - value_393 = 393; - value_394 = 394; - value_395 = 395; - value_396 = 396; - value_397 = 397; - value_398 = 398; - value_399 = 399; - value_400 = 400; - value_401 = 401; - value_402 = 402; - value_403 = 403; - value_404 = 404; - value_405 = 405; - value_406 = 406; - value_407 = 407; - value_408 = 408; - value_409 = 409; - value_410 = 410; - value_411 = 411; - value_412 = 412; - value_413 = 413; - value_414 = 414; - value_415 = 415; - value_416 = 416; - value_417 = 417; - value_418 = 418; - value_419 = 419; - value_420 = 420; - value_421 = 421; - value_422 = 422; - value_423 = 423; - value_424 = 424; - value_425 = 425; - value_426 = 426; - value_427 = 427; - value_428 = 428; - value_429 = 429; - value_430 = 430; - value_431 = 431; - value_432 = 432; - value_433 = 433; - value_434 = 434; - value_435 = 435; - value_436 = 436; - value_437 = 437; - value_438 = 438; - value_439 = 439; - value_440 = 440; - value_441 = 441; - value_442 = 442; - value_443 = 443; - value_444 = 444; - value_445 = 445; - value_446 = 446; - value_447 = 447; - value_448 = 448; - value_449 = 449; - value_450 = 450; - value_451 = 451; - value_452 = 452; - value_453 = 453; - value_454 = 454; - value_455 = 455; - value_456 = 456; - value_457 = 457; - value_458 = 458; - value_459 = 459; - value_460 = 460; - value_461 = 461; - value_462 = 462; - value_463 = 463; - value_464 = 464; - value_465 = 465; - value_466 = 466; - value_467 = 467; - value_468 = 468; - value_469 = 469; - value_470 = 470; - value_471 = 471; - value_472 = 472; - value_473 = 473; - value_474 = 474; - value_475 = 475; - value_476 = 476; - value_477 = 477; - value_478 = 478; - value_479 = 479; - value_480 = 480; - value_481 = 481; - value_482 = 482; - value_483 = 483; - value_484 = 484; - value_485 = 485; - value_486 = 486; - value_487 = 487; - value_488 = 488; - value_489 = 489; - value_490 = 490; - value_491 = 491; - value_492 = 492; - value_493 = 493; - value_494 = 494; - value_495 = 495; - value_496 = 496; - value_497 = 497; - value_498 = 498; - value_499 = 499; - value_500 = 500; - value_501 = 501; - value_502 = 502; - value_503 = 503; - value_504 = 504; - value_505 = 505; - value_506 = 506; - value_507 = 507; - } - Value value = 1; - Sentence inner = 2; -} -message Sentence { - repeated Word words = 1; -} \ No newline at end of file From be28e94471f4342e7b73b01b15b56b7b98392f45 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 06:30:56 +0530 Subject: [PATCH 475/950] add compile flag -force-vector-width=16 due to llvm autovectorization bug with avx512 --- cmake/cpu_features.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 4ea9465be98..e77f330fbc0 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -129,8 +129,8 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () - - set (TEST_FLAG "-mavx512f -mavx512bw") +#we have to add -force-vector-width=16 due to llvm autovectorization bug with avx512 + set (TEST_FLAG "-mavx512f -mavx512bw -force-vector-width=16") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include From 5ba876cad29686a3c8c0aee68b65219223b2f140 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 05:46:19 +0300 Subject: [PATCH 476/950] Build protoc for host architecture during cross-compilation --- .../grpc-cmake/protobuf_generate_grpc.cmake | 4 +- contrib/protobuf-cmake/CMakeLists.txt | 49 ++++++++++++++++++- .../protobuf-cmake/protobuf_generate.cmake | 4 +- 3 files changed, 52 insertions(+), 5 deletions(-) diff --git a/contrib/grpc-cmake/protobuf_generate_grpc.cmake b/contrib/grpc-cmake/protobuf_generate_grpc.cmake index 726428a7597..71ee69caf3e 100644 --- a/contrib/grpc-cmake/protobuf_generate_grpc.cmake +++ b/contrib/grpc-cmake/protobuf_generate_grpc.cmake @@ -187,12 +187,12 @@ function(protobuf_generate_grpc) add_custom_command( OUTPUT ${_generated_srcs} - COMMAND $ + COMMAND $ ARGS --${protobuf_generate_grpc_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} --grpc_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} --plugin=protoc-gen-grpc=$ ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} - DEPENDS ${_abs_file} protobuf::protoc ${protobuf_generate_grpc_PLUGIN} + DEPENDS ${_abs_file} protoc ${protobuf_generate_grpc_PLUGIN} COMMENT "Running ${protobuf_generate_grpc_LANGUAGE} protocol buffer compiler on ${_proto}" VERBATIM) endforeach() diff --git a/contrib/protobuf-cmake/CMakeLists.txt b/contrib/protobuf-cmake/CMakeLists.txt index a4993030d04..10ef7df26b1 100644 --- a/contrib/protobuf-cmake/CMakeLists.txt +++ b/contrib/protobuf-cmake/CMakeLists.txt @@ -10,8 +10,55 @@ else () set(protobuf_BUILD_SHARED_LIBS ON CACHE INTERNAL "" FORCE) endif () +if (CMAKE_CROSSCOMPILING) + # Will build 'protoc' for host arch instead of cross-compiling + set(protobuf_BUILD_PROTOC_BINARIES OFF CACHE INTERNAL "" FORCE) +endif () + add_subdirectory("${protobuf_SOURCE_DIR}/cmake" "${protobuf_BINARY_DIR}") # We don't want to stop compilation on warnings in protobuf's headers. -# The following line overrides the value assigned by the command target_include_directories() in libprotobuf.cmake +# The following line overrides the value assigned by the command target_include_directories() in libprotobuf.cmake set_property(TARGET libprotobuf PROPERTY INTERFACE_SYSTEM_INCLUDE_DIRECTORIES "${protobuf_SOURCE_DIR}/src") + +if (CMAKE_CROSSCOMPILING) + # Build 'protoc' for host arch + set (PROTOC_BUILD_DIR "${protobuf_BINARY_DIR}/build") + + add_custom_command ( + OUTPUT ${PROTOC_BUILD_DIR} + COMMAND mkdir -p ${PROTOC_BUILD_DIR}) + + add_custom_command ( + OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" + + COMMAND ${CMAKE_COMMAND} + -G"${CMAKE_GENERATOR}" + -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" + -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" + -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" + -Dprotobuf_BUILD_TESTS=0 + -Dprotobuf_BUILD_CONFORMANCE=0 + -Dprotobuf_BUILD_EXAMPLES=0 + -Dprotobuf_BUILD_PROTOC_BINARIES=1 + "${protobuf_SOURCE_DIR}/cmake" + + DEPENDS "${PROTOC_BUILD_DIR}" + WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" + COMMENT "Configuring 'protoc' for host architecture.") + + add_custom_command ( + OUTPUT "${PROTOC_BUILD_DIR}/protoc" + COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" + DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" + COMMENT "Building 'protoc' for host architecture.") + + #add_custom_target (protoc DEPENDS "${PROTOC_BUILD_DIR}/protoc") + #set_target_properties (protoc PROPERTIES TARGET_FILE "${PROTOC_BUILD_DIR}/protoc") + + add_executable(protoc IMPORTED GLOBAL) + set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") + add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") + + #add_executable(protobuf::protoc ALIAS protoc) +endif () diff --git a/contrib/protobuf-cmake/protobuf_generate.cmake b/contrib/protobuf-cmake/protobuf_generate.cmake index c444162dd1e..3e30b4e40fd 100644 --- a/contrib/protobuf-cmake/protobuf_generate.cmake +++ b/contrib/protobuf-cmake/protobuf_generate.cmake @@ -181,9 +181,9 @@ function(protobuf_generate) add_custom_command( OUTPUT ${_generated_srcs} - COMMAND $ + COMMAND $ ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} - DEPENDS ${_abs_file} protobuf::protoc + DEPENDS ${_abs_file} protoc COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" VERBATIM) endforeach() From 12168e7762712f9e68544d49076e34befa11affd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 05:46:50 +0300 Subject: [PATCH 477/950] Enable Protobuf, Arrow, ORC, Parquet for AArch64 and Darwin --- cmake/target.cmake | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/cmake/target.cmake b/cmake/target.cmake index d1a0b8f9cbf..683deab1c33 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -20,24 +20,14 @@ endif () if (CMAKE_CROSSCOMPILING) if (OS_DARWIN) # FIXME: broken dependencies - set (ENABLE_PROTOBUF OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") # no protobuf -> no grpc - set (USE_SNAPPY OFF CACHE INTERNAL "") - set (ENABLE_PARQUET OFF CACHE INTERNAL "") # no snappy and protobuf -> no parquet - set (ENABLE_ORC OFF CACHE INTERNAL "") # no arrow (parquet) -> no orc - set (ENABLE_ICU OFF CACHE INTERNAL "") set (ENABLE_FASTOPS OFF CACHE INTERNAL "") elseif (OS_LINUX OR OS_ANDROID) if (ARCH_AARCH64) # FIXME: broken dependencies - set (ENABLE_PROTOBUF OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") - - set (ENABLE_PARQUET OFF CACHE INTERNAL "") - set (ENABLE_ORC OFF CACHE INTERNAL "") - set (ENABLE_MYSQL OFF CACHE INTERNAL "") endif () elseif (OS_FREEBSD) From 3be83a26d00dcfa4712627fc57a08169919a34d6 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 08:24:47 +0530 Subject: [PATCH 478/950] fixed Typos and Styles issues --- src/CMakeLists.txt | 2 +- src/Columns/ColumnFixedString.cpp | 8 ++++---- src/Columns/ColumnVector.cpp | 7 +++---- src/Columns/ColumnsCommon.cpp | 18 +++++++++--------- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 45bb1a21d59..d5977b31159 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,7 +284,7 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) -# Use X86 AVX2/AVX512 instructions to accelerate filter opertions +# Use X86 AVX2/AVX512 instructions to accelerate filter operations set_source_files_properties( Columns/ColumnFixedString.cpp Columns/ColumnsCommon.cpp diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 4b31677d37e..65f56495445 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -238,13 +238,13 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; - const __m512i zero64 = _mm512_setzero_epi32(); + const __m512i zero64 = _mm512_setzero_epi32(); const UInt8 * filt_end_avx512 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; const size_t chars_per_simd_elements = SIMD_BYTES * n; while (filt_pos < filt_end_avx512) - { - uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); if (0xFFFFFFFFFFFFFFFF == mask) { @@ -263,7 +263,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result mask = _blsr_u64(mask); #else mask = mask & (mask-1); - #endif + #endif } } data_pos += chars_per_simd_elements; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 37f60e9f2b9..78922841e37 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -310,7 +310,6 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); - /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -318,11 +317,11 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; - const __m512i zero64 = _mm512_setzero_epi32(); + const __m512i zero64 = _mm512_setzero_epi32(); const UInt8 * filt_end_avx512 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_avx512) - { + { UInt64 mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); if (0xFFFFFFFFFFFFFFFF == mask) @@ -339,7 +338,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s mask = _blsr_u64(mask); #else mask = mask & (mask-1); - #endif + #endif } } diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 36c292b4196..d8e0cf74669 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -235,8 +235,8 @@ namespace const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_aligned) - { - uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); if (mask == 0xffffffffffffffff) { @@ -258,18 +258,18 @@ namespace while (mask) { size_t index = __builtin_ctzll(mask); - copy_array(offsets_pos + index); + copy_array(offsets_pos + index); #ifdef __BMI__ mask = _blsr_u64(mask); #else mask = mask & (mask-1); - #endif + #endif } } filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #elif defined(__AVX2__) const __m256i zero_vec = _mm256_setzero_si256(); static constexpr size_t SIMD_BYTES = 32; @@ -277,7 +277,7 @@ namespace while (filt_pos < filt_end_aligned) { - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); if (mask == 0xffffffff) { @@ -299,18 +299,18 @@ namespace while (mask) { size_t index = __builtin_ctz(mask); - copy_array(offsets_pos + index); + copy_array(offsets_pos + index); #ifdef __BMI__ mask = _blsr_u32(mask); #else mask = mask & (mask-1); - #endif + #endif } } filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #elif defined(__SSE2__) const __m128i zero_vec = _mm_setzero_si128(); static constexpr size_t SIMD_BYTES = 16; From 3a8e65f01e31baa24c6ccc0c81168a128c679b61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 06:28:05 +0300 Subject: [PATCH 479/950] Another try --- contrib/protobuf-cmake/CMakeLists.txt | 79 ++++++++++++++++++--------- 1 file changed, 53 insertions(+), 26 deletions(-) diff --git a/contrib/protobuf-cmake/CMakeLists.txt b/contrib/protobuf-cmake/CMakeLists.txt index 10ef7df26b1..0ceb72cfbd6 100644 --- a/contrib/protobuf-cmake/CMakeLists.txt +++ b/contrib/protobuf-cmake/CMakeLists.txt @@ -25,40 +25,67 @@ if (CMAKE_CROSSCOMPILING) # Build 'protoc' for host arch set (PROTOC_BUILD_DIR "${protobuf_BINARY_DIR}/build") - add_custom_command ( - OUTPUT ${PROTOC_BUILD_DIR} - COMMAND mkdir -p ${PROTOC_BUILD_DIR}) + if (NOT EXISTS "${PROTOC_BUILD_DIR}/protoc") - add_custom_command ( - OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" + # This is quite ugly but I cannot make dependencies work propery. - COMMAND ${CMAKE_COMMAND} - -G"${CMAKE_GENERATOR}" - -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" - -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" - -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" - -Dprotobuf_BUILD_TESTS=0 - -Dprotobuf_BUILD_CONFORMANCE=0 - -Dprotobuf_BUILD_EXAMPLES=0 - -Dprotobuf_BUILD_PROTOC_BINARIES=1 - "${protobuf_SOURCE_DIR}/cmake" + execute_process( + COMMAND mkdir -p ${PROTOC_BUILD_DIR} + COMMAND_ECHO STDOUT) - DEPENDS "${PROTOC_BUILD_DIR}" - WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" - COMMENT "Configuring 'protoc' for host architecture.") + execute_process( + COMMAND ${CMAKE_COMMAND} + "-G${CMAKE_GENERATOR}" + "-DCMAKE_MAKE_PROGRAM=${CMAKE_MAKE_PROGRAM}" + "-DCMAKE_C_COMPILER=${CMAKE_C_COMPILER}" + "-DCMAKE_CXX_COMPILER=${CMAKE_CXX_COMPILER}" + "-Dprotobuf_BUILD_TESTS=0" + "-Dprotobuf_BUILD_CONFORMANCE=0" + "-Dprotobuf_BUILD_EXAMPLES=0" + "-Dprotobuf_BUILD_PROTOC_BINARIES=1" + "${protobuf_SOURCE_DIR}/cmake" + WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" + COMMAND_ECHO STDOUT) - add_custom_command ( - OUTPUT "${PROTOC_BUILD_DIR}/protoc" - COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" - DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" - COMMENT "Building 'protoc' for host architecture.") + execute_process( + COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" + COMMAND_ECHO STDOUT) + endif () - #add_custom_target (protoc DEPENDS "${PROTOC_BUILD_DIR}/protoc") - #set_target_properties (protoc PROPERTIES TARGET_FILE "${PROTOC_BUILD_DIR}/protoc") +# add_custom_command ( +# OUTPUT ${PROTOC_BUILD_DIR} +# COMMAND mkdir -p ${PROTOC_BUILD_DIR}) +# +# add_custom_command ( +# OUTPUT "${PROTOC_BUILD_DIR}/CMakeCache.txt" +# +# COMMAND ${CMAKE_COMMAND} +# -G"${CMAKE_GENERATOR}" +# -DCMAKE_MAKE_PROGRAM="${CMAKE_MAKE_PROGRAM}" +# -DCMAKE_C_COMPILER="${CMAKE_C_COMPILER}" +# -DCMAKE_CXX_COMPILER="${CMAKE_CXX_COMPILER}" +# -Dprotobuf_BUILD_TESTS=0 +# -Dprotobuf_BUILD_CONFORMANCE=0 +# -Dprotobuf_BUILD_EXAMPLES=0 +# -Dprotobuf_BUILD_PROTOC_BINARIES=1 +# "${protobuf_SOURCE_DIR}/cmake" +# +# DEPENDS "${PROTOC_BUILD_DIR}" +# WORKING_DIRECTORY "${PROTOC_BUILD_DIR}" +# COMMENT "Configuring 'protoc' for host architecture." +# USES_TERMINAL) +# +# add_custom_command ( +# OUTPUT "${PROTOC_BUILD_DIR}/protoc" +# COMMAND ${CMAKE_COMMAND} --build "${PROTOC_BUILD_DIR}" +# DEPENDS "${PROTOC_BUILD_DIR}/CMakeCache.txt" +# COMMENT "Building 'protoc' for host architecture." +# USES_TERMINAL) +# +# add_custom_target (protoc-host DEPENDS "${PROTOC_BUILD_DIR}/protoc") add_executable(protoc IMPORTED GLOBAL) set_target_properties (protoc PROPERTIES IMPORTED_LOCATION "${PROTOC_BUILD_DIR}/protoc") add_dependencies(protoc "${PROTOC_BUILD_DIR}/protoc") - #add_executable(protobuf::protoc ALIAS protoc) endif () From 8f349907fb2fad802d9ceeb9cab7147d71fb8deb Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 09:21:54 +0530 Subject: [PATCH 480/950] Fixed trailing whitespaces issue --- src/Columns/ColumnFixedString.cpp | 8 ++++---- src/Columns/ColumnVector.cpp | 6 +++--- src/Columns/ColumnsCommon.cpp | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 65f56495445..a84db5701a7 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -235,7 +235,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. - */ + */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; const __m512i zero64 = _mm512_setzero_epi32(); @@ -271,13 +271,13 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result } #elif defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; - const __m256i zero32 = _mm256_setzero_si256(); + const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; const size_t chars_per_simd_elements = SIMD_BYTES * n; while (filt_pos < filt_end_avx2) { - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); if (0xFFFFFFFF == mask) { @@ -296,7 +296,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result mask = _blsr_u32(mask); #else mask = mask & (mask-1); - #endif + #endif } } data_pos += chars_per_simd_elements; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 78922841e37..6bc596008d8 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -348,12 +348,12 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s #elif defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; - const __m256i zero32 = _mm256_setzero_si256(); + const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_avx2) { - UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); if (0xFFFFFFFF == mask) { @@ -369,7 +369,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s mask = _blsr_u32(mask); #else mask = mask & (mask-1); - #endif + #endif } } diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index d8e0cf74669..845bac79ae0 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -237,7 +237,7 @@ namespace while (filt_pos < filt_end_aligned) { uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); - + if (mask == 0xffffffffffffffff) { /// SIMD_BYTES consecutive rows pass the filter @@ -350,7 +350,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #endif while (filt_pos < filt_end) From b5d69d599e3b9f7fd8654a8e5cbd2d1dba374273 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 12 Oct 2021 04:13:02 +0000 Subject: [PATCH 481/950] fix sample by tuple() add tests fix fix fix --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++++ tests/queries/0_stateless/02096_sample_by_tuple.reference | 0 tests/queries/0_stateless/02096_sample_by_tuple.sql | 7 +++++++ 3 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02096_sample_by_tuple.reference create mode 100644 tests/queries/0_stateless/02096_sample_by_tuple.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c04e0d2e38f..51b68eed951 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -132,10 +132,14 @@ namespace ErrorCodes extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; extern const int SUPPORT_IS_DISABLED; extern const int TOO_MANY_SIMULTANEOUS_QUERIES; + extern const int INCORRECT_QUERY; } static void checkSampleExpression(const StorageInMemoryMetadata & metadata, bool allow_sampling_expression_not_in_primary_key, bool check_sample_column_is_correct) { + if (metadata.sampling_key.column_names.empty()) + throw Exception("There are no columns in sampling expression", ErrorCodes::INCORRECT_QUERY); + const auto & pk_sample_block = metadata.getPrimaryKey().sample_block; if (!pk_sample_block.has(metadata.sampling_key.column_names[0]) && !allow_sampling_expression_not_in_primary_key) throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); diff --git a/tests/queries/0_stateless/02096_sample_by_tuple.reference b/tests/queries/0_stateless/02096_sample_by_tuple.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02096_sample_by_tuple.sql b/tests/queries/0_stateless/02096_sample_by_tuple.sql new file mode 100644 index 00000000000..4996c9b8384 --- /dev/null +++ b/tests/queries/0_stateless/02096_sample_by_tuple.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS t; + +CREATE TABLE t (n UInt8) ENGINE=MergeTree ORDER BY n SAMPLE BY tuple(); -- { serverError 80 } + +CREATE TABLE t (n UInt8) ENGINE=MergeTree ORDER BY tuple(); + +ALTER TABLE t MODIFY SAMPLE BY tuple(); -- { serverError 80 } From ee1fd495ea77015db254a42108b630175c5930af Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 12 Oct 2021 05:30:35 +0000 Subject: [PATCH 482/950] remove redundant dot in exception message --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 3965945b3ca..c098c6e0506 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -999,7 +999,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } else throw Exception(storage_already_exists_error_code, - "{} {}.{} already exists.", storage_name, backQuoteIfNeed(create.database), backQuoteIfNeed(create.table)); + "{} {}.{} already exists", storage_name, backQuoteIfNeed(create.database), backQuoteIfNeed(create.table)); } data_path = database->getTableDataPath(create); From 88b18074348b8262625358eaebdf26c06cc8b8d7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Oct 2021 10:33:45 +0300 Subject: [PATCH 483/950] Fix special build. --- src/Formats/registerFormats.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index a98876c15bd..3e4c0366e8a 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -54,8 +54,6 @@ void registerOutputFormatRawBLOB(FormatFactory & factory); /// Output only (presentational) formats. -void registerOutputFormatNull(FormatFactory & factory); - void registerOutputFormatPretty(FormatFactory & factory); void registerOutputFormatPrettyCompact(FormatFactory & factory); void registerOutputFormatPrettySpace(FormatFactory & factory); From 5a0f22fbce602d6f09a8e50771a8a2fef1a9b0a0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Oct 2021 11:27:18 +0300 Subject: [PATCH 484/950] Fix resolve host --- src/IO/S3/PocoHTTPClient.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 1a23e95d648..68bdbc9cf86 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -173,7 +173,7 @@ void PocoHTTPClient::makeRequestInternal( { /// Reverse proxy can replace host header with resolved ip address instead of host name. /// This can lead to request signature difference on S3 side. - session = makeHTTPSession(target_uri, timeouts, false); + session = makeHTTPSession(target_uri, timeouts, /* resolve_host = */ false); bool use_tunnel = request_configuration.proxyScheme == Aws::Http::Scheme::HTTP && target_uri.getScheme() == "https"; session->setProxy( @@ -185,7 +185,7 @@ void PocoHTTPClient::makeRequestInternal( } else { - session = makeHTTPSession(target_uri, timeouts, true); + session = makeHTTPSession(target_uri, timeouts, /* resolve_host = */ true); } From c15b67c18264919df7c8048ea36ea5058a185d36 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Oct 2021 11:42:24 +0300 Subject: [PATCH 485/950] Fix naming --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1ede7669832..f9c26225440 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3006,7 +3006,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D for (const auto & column : part->getColumns()) { ColumnSize & total_column_size = column_sizes[column.name]; - ColumnSize part_secondary_index_size = part->getColumnSize(column.name, *column.type); + ColumnSize part_column_size = part->getColumnSize(column.name, *column.type); auto log_subtract = [&](size_t & from, size_t value, const char * field) { @@ -3017,9 +3017,9 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D from -= value; }; - log_subtract(total_column_size.data_compressed, part_secondary_index_size.data_compressed, ".data_compressed"); - log_subtract(total_column_size.data_uncompressed, part_secondary_index_size.data_uncompressed, ".data_uncompressed"); - log_subtract(total_column_size.marks, part_secondary_index_size.marks, ".marks"); + log_subtract(total_column_size.data_compressed, part_column_size.data_compressed, ".data_compressed"); + log_subtract(total_column_size.data_uncompressed, part_column_size.data_uncompressed, ".data_uncompressed"); + log_subtract(total_column_size.marks, part_column_size.marks, ".marks"); } auto indexes_descriptions = getInMemoryMetadataPtr()->secondary_indices; From cb176cf9addf71b94b58cbe23b2c542d7417896d Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 12 Oct 2021 12:33:54 +0300 Subject: [PATCH 486/950] Add `long` tag to 02033_join_engine_deadlock_long --- tests/queries/0_stateless/02033_join_engine_deadlock_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh index f4ae564e2a7..2a887cbbcae 100755 --- a/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh +++ b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock +# Tags: long, deadlock CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 91afa20a737f5844bc37606fe9e758d02195f5d8 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Fri, 8 Oct 2021 20:00:09 +0300 Subject: [PATCH 487/950] Unit tests for dictionary with comment --- .../tests/gtest_dictionary_configuration.cpp | 6 +++- src/Parsers/tests/gtest_Parser.cpp | 36 ++++++++++++++++--- 2 files changed, 36 insertions(+), 6 deletions(-) diff --git a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp index 8542a43e202..6e97910c6e1 100644 --- a/src/Dictionaries/tests/gtest_dictionary_configuration.cpp +++ b/src/Dictionaries/tests/gtest_dictionary_configuration.cpp @@ -44,7 +44,8 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration) " SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' PASSWORD '' DB 'test' TABLE 'table_for_dict'))" " LAYOUT(FLAT())" " LIFETIME(MIN 1 MAX 10)" - " RANGE(MIN second_column MAX third_column)"; + " RANGE(MIN second_column MAX third_column)" + " COMMENT 'hello world!'"; ParserCreateDictionaryQuery parser; ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); @@ -92,6 +93,9 @@ TEST(ConvertDictionaryAST, SimpleDictConfiguration) /// layout EXPECT_TRUE(config->has("dictionary.layout.flat")); + + // comment + EXPECT_TRUE(config->has("dictionary.comment")); } diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index f97bc77272c..5ebea834a91 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -132,11 +133,6 @@ INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest, "MODIFY COMMENT ''", "MODIFY COMMENT ''", }, -// { -// // No comment - same as empty comment -// "MODIFY COMMENT NULL", -// "MODIFY COMMENT ''", -// }, { // Non-empty comment value "MODIFY COMMENT 'some comment value'", @@ -144,3 +140,33 @@ INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest, } } ))); + + +INSTANTIATE_TEST_SUITE_P(ParserCreateQuery_DICTIONARY_WITH_COMMENT, ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list{ + { + R"sql(CREATE DICTIONARY 2024_dictionary_with_comment +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) +LAYOUT(FLAT()) +LIFETIME(MIN 0 MAX 1000) +COMMENT 'Test dictionary with comment'; +)sql", + R"sql(CREATE DICTIONARY `2024_dictionary_with_comment` +( + `id` UInt64, + `value` String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) +LIFETIME(MIN 0 MAX 1000) +LAYOUT(FLAT()) +COMMENT 'Test dictionary with comment')sql" + }} +))); From b2f8cd07e68107db07ffe79abdf7948d48f2f90d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 12 Oct 2021 12:54:02 +0300 Subject: [PATCH 488/950] Update normalizeString.cpp --- src/Functions/normalizeString.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/normalizeString.cpp b/src/Functions/normalizeString.cpp index 5beca566cd1..2b3a869e452 100644 --- a/src/Functions/normalizeString.cpp +++ b/src/Functions/normalizeString.cpp @@ -10,9 +10,9 @@ #include #include #include -#include "common/logger_useful.h" -#include "Columns/ColumnString.h" -#include "Parsers/IAST_fwd.h" +#include +#include +#include namespace DB { From 93b3fcc19533962c47f22dbfd8535f3b87412ce5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Oct 2021 13:05:20 +0300 Subject: [PATCH 489/950] Fix rabbitmq --- src/Storages/RabbitMQ/RabbitMQSource.cpp | 5 +++-- tests/integration/test_storage_rabbitmq/test.py | 4 +++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 2ea60bfb68b..34edd06d3e2 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -118,8 +119,8 @@ Chunk RabbitMQSource::generateImpl() is_finished = true; MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); - auto input_format = context->getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, max_block_size); + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); StreamingFormatExecutor executor(non_virtual_header, input_format); diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 9e2752438f8..85dda2fe4d3 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -463,11 +463,13 @@ def test_rabbitmq_big_message(rabbitmq_cluster): for message in messages: channel.basic_publish(exchange='big', routing_key='', body=message) - while True: + for _ in range(300): result = instance.query('SELECT count() FROM test.view') if int(result) == batch_messages * rabbitmq_messages: break + time.sleep(1) + connection.close() instance.query(''' DROP TABLE test.consumer; From 5bf64c62c271cef95f5cfda1796eef82311c6bdd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 12 Oct 2021 10:19:21 +0000 Subject: [PATCH 490/950] Delete test --- src/Functions/tests/gtest_abtesting.cpp | 105 ------------------------ 1 file changed, 105 deletions(-) delete mode 100644 src/Functions/tests/gtest_abtesting.cpp diff --git a/src/Functions/tests/gtest_abtesting.cpp b/src/Functions/tests/gtest_abtesting.cpp deleted file mode 100644 index e7ef5b5c3cf..00000000000 --- a/src/Functions/tests/gtest_abtesting.cpp +++ /dev/null @@ -1,105 +0,0 @@ -#include - -#if !defined(ARCADIA_BUILD) && USE_STATS - -# include - -using namespace DB; - -Variants test_bayesab(std::string dist, PODArray xs, PODArray ys, size_t & max, size_t & min) -{ - Variants variants; - - //std::cout << std::fixed; - if (dist == "beta") - { -/* std::cout << dist << "\nclicks: "; - for (auto x : xs) - std::cout << x << " "; - - std::cout <<"\tconversions: "; - for (auto y : ys) - std::cout << y << " "; - - std::cout << "\n";*/ - - variants = bayesian_ab_test(dist, xs, ys); - } - else if (dist == "gamma") - { -/* std::cout << dist << "\nclicks: "; - for (auto x : xs) - std::cout << x << " "; - - std::cout <<"\tcost: "; - for (auto y : ys) - std::cout << y << " "; - - std::cout << "\n";*/ - - variants = bayesian_ab_test(dist, xs, ys); - } - -/* for (size_t i = 0; i < variants.size(); ++i) - std::cout << i << " beats 0: " << variants[i].beats_control << std::endl; - - for (size_t i = 0; i < variants.size(); ++i) - std::cout << i << " to be best: " << variants[i].best << std::endl; - - std::cout << convertToJson({"0", "1", "2"}, variants) << std::endl; -*/ - Float64 max_val = 0.0, min_val = 2.0; - for (size_t i = 0; i < variants.size(); ++i) - { - if (variants[i].best > max_val) - { - max_val = variants[i].best; - max = i; - } - - if (variants[i].best < min_val) - { - min_val = variants[i].best; - min = i; - } - } - - return variants; -} - - -TEST(BayesAB, beta) -{ - size_t max = 0, min = 0; - - auto variants = test_bayesab("beta", {10000, 1000, 900}, {600, 110, 90}, max, min); - ASSERT_EQ(1, max); - - variants = test_bayesab("beta", {3000, 3000, 3000}, {600, 100, 90}, max, min); - ASSERT_EQ(0, max); - - variants = test_bayesab("beta", {3000, 3000, 3000}, {100, 90, 110}, max, min); - ASSERT_EQ(2, max); - - variants = test_bayesab("beta", {3000, 3000, 3000}, {110, 90, 100}, max, min); - ASSERT_EQ(0, max); -} - - -TEST(BayesAB, gamma) -{ - size_t max = 0, min = 0; - auto variants = test_bayesab("gamma", {10000, 1000, 900}, {600, 110, 90}, max, min); - ASSERT_EQ(1, max); - - variants = test_bayesab("gamma", {3000, 3000, 3000}, {600, 100, 90}, max, min); - ASSERT_EQ(0, max); - - variants = test_bayesab("gamma", {3000, 3000, 3000}, {100, 90, 110}, max, min); - ASSERT_EQ(2, max); - - variants = test_bayesab("gamma", {3000, 3000, 3000}, {110, 90, 100}, max, min); - ASSERT_EQ(0, max); -} - -#endif From a33610aa3259d5a45c25a1a4d02204855feb9241 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Oct 2021 18:56:23 +0300 Subject: [PATCH 491/950] Make read of Counters snapshot non-atomic --- src/Common/ProfileEvents.cpp | 10 +++++++--- src/Common/ProfileEvents.h | 20 +++++++++++++++++++- src/Interpreters/ProcessList.cpp | 4 ++-- src/Interpreters/ProcessList.h | 4 ++-- src/Interpreters/ProfileEventsExt.cpp | 4 ++-- src/Interpreters/ProfileEventsExt.h | 2 +- src/Interpreters/QueryLog.h | 2 +- src/Interpreters/QueryThreadLog.h | 2 +- src/Interpreters/QueryViewsLog.h | 2 +- src/Interpreters/ThreadStatusExt.cpp | 4 ++-- 10 files changed, 38 insertions(+), 16 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fa23f4df533..204a2d889c6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -301,11 +301,15 @@ void Counters::reset() resetCounters(); } -Counters Counters::getPartiallyAtomicSnapshot() const +Counters::Snapshot::Snapshot() + : counters_holder(new Count[num_counters] {}) +{} + +Counters::Snapshot Counters::getPartiallyAtomicSnapshot() const { - Counters res(VariableContext::Snapshot, nullptr); + Snapshot res; for (Event i = 0; i < num_counters; ++i) - res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed); + res[i] = counters[i].load(std::memory_order_relaxed); return res; } diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index e1b68e43e52..12d057a533f 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -59,8 +59,26 @@ namespace ProfileEvents } while (current != nullptr); } + struct Snapshot + { + Snapshot(); + + Count & operator[] (Event event) + { + return counters_holder[event]; + } + + const Count & operator[] (Event event) const + { + return counters_holder[event]; + } + + private: + std::unique_ptr counters_holder; + }; + /// Every single value is fetched atomically, but not all values as a whole. - Counters getPartiallyAtomicSnapshot() const; + Snapshot getPartiallyAtomicSnapshot() const; /// Reset all counters to zero and reset parent. void reset(); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index f8402cf0287..5a77ebb1dfe 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -408,7 +408,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even } if (get_profile_events) - res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); + res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); } if (get_settings && getContext()) @@ -446,7 +446,7 @@ ProcessListForUserInfo ProcessListForUser::getInfo(bool get_profile_events) cons res.peak_memory_usage = user_memory_tracker.getPeak(); if (get_profile_events) - res.profile_counters = std::make_shared(user_performance_counters.getPartiallyAtomicSnapshot()); + res.profile_counters = std::make_shared(user_performance_counters.getPartiallyAtomicSnapshot()); return res; } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 2e300472647..e0a52772da7 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -66,7 +66,7 @@ struct QueryStatusInfo /// Optional fields, filled by query std::vector thread_ids; - std::shared_ptr profile_counters; + std::shared_ptr profile_counters; std::shared_ptr query_settings; std::string current_database; }; @@ -186,7 +186,7 @@ struct ProcessListForUserInfo Int64 peak_memory_usage; // Optional field, filled by request. - std::shared_ptr profile_counters; + std::shared_ptr profile_counters; }; diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 2e8f986ca6c..4386c294316 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -12,7 +12,7 @@ namespace ProfileEvents { /// Put implementation here to avoid extra linking dependencies for clickhouse_common_io -void dumpToMapColumn(const Counters & counters, DB::IColumn * column, bool nonzero_only) +void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only) { auto * column_map = column ? &typeid_cast(*column) : nullptr; if (!column_map) @@ -26,7 +26,7 @@ void dumpToMapColumn(const Counters & counters, DB::IColumn * column, bool nonze size_t size = 0; for (Event event = 0; event < Counters::num_counters; ++event) { - UInt64 value = counters[event].load(std::memory_order_relaxed); + UInt64 value = counters[event]; if (nonzero_only && 0 == value) continue; diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index 7d513f0cd02..699c997d904 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -7,6 +7,6 @@ namespace ProfileEvents { /// Dumps profile events to columns Map(String, UInt64) -void dumpToMapColumn(const Counters & counters, DB::IColumn * column, bool nonzero_only = true); +void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true); } diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 2713febe1b6..49c38e7d2a9 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -80,7 +80,7 @@ struct QueryLogElement String log_comment; std::vector thread_ids; - std::shared_ptr profile_counters; + std::shared_ptr profile_counters; std::shared_ptr query_settings; static std::string name() { return "QueryLog"; } diff --git a/src/Interpreters/QueryThreadLog.h b/src/Interpreters/QueryThreadLog.h index 57e93edbaf7..f826ebac4fd 100644 --- a/src/Interpreters/QueryThreadLog.h +++ b/src/Interpreters/QueryThreadLog.h @@ -45,7 +45,7 @@ struct QueryThreadLogElement ClientInfo client_info; - std::shared_ptr profile_counters; + std::shared_ptr profile_counters; static std::string name() { return "QueryThreadLog"; } diff --git a/src/Interpreters/QueryViewsLog.h b/src/Interpreters/QueryViewsLog.h index a84e9f9ba89..5b0567c3b60 100644 --- a/src/Interpreters/QueryViewsLog.h +++ b/src/Interpreters/QueryViewsLog.h @@ -64,7 +64,7 @@ struct QueryViewsLogElement UInt64 written_rows{}; UInt64 written_bytes{}; Int64 peak_memory_usage{}; - std::shared_ptr profile_counters; + std::shared_ptr profile_counters; ViewStatus status = ViewStatus::QUERY_START; Int32 exception_code{}; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 7ff74a0618c..fff803fa559 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -478,7 +478,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String if (query_context_ptr->getSettingsRef().log_profile_events != 0) { /// NOTE: Here we are in the same thread, so we can make memcpy() - elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + elem.profile_counters = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); } } @@ -519,7 +519,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr); element.view_target = vinfo.runtime_stats->target_name; - auto events = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); + auto events = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); element.read_bytes = progress_in.read_bytes.load(std::memory_order_relaxed); element.written_rows = (*events)[ProfileEvents::InsertedRows]; From 857dd528673cd7235c7fa54b689077e1cdba2a3c Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Oct 2021 22:30:56 +0300 Subject: [PATCH 492/950] Remove redundant enumerator --- src/Common/VariableContext.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/VariableContext.h b/src/Common/VariableContext.h index 2fe4ffb565a..fb9acd40aea 100644 --- a/src/Common/VariableContext.h +++ b/src/Common/VariableContext.h @@ -8,5 +8,4 @@ enum class VariableContext User, /// Group of processes Process, /// For example, a query or a merge Thread, /// A thread of a process - Snapshot /// Does not belong to anybody }; From abb74d2a5b594c9bd76a664911fb87e1c5ff508e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 11 Oct 2021 22:45:47 +0300 Subject: [PATCH 493/950] Make Counters::Snapshot immutable --- src/Common/ProfileEvents.cpp | 2 +- src/Common/ProfileEvents.h | 7 ++----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 204a2d889c6..941a3ab0896 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -309,7 +309,7 @@ Counters::Snapshot Counters::getPartiallyAtomicSnapshot() const { Snapshot res; for (Event i = 0; i < num_counters; ++i) - res[i] = counters[i].load(std::memory_order_relaxed); + res.counters_holder[i] = counters[i].load(std::memory_order_relaxed); return res; } diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 12d057a533f..c416b49dd5c 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -63,11 +63,6 @@ namespace ProfileEvents { Snapshot(); - Count & operator[] (Event event) - { - return counters_holder[event]; - } - const Count & operator[] (Event event) const { return counters_holder[event]; @@ -75,6 +70,8 @@ namespace ProfileEvents private: std::unique_ptr counters_holder; + + friend class Counters; }; /// Every single value is fetched atomically, but not all values as a whole. From 8be70bc417397de270e08cbe64a90be278972b81 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 12 Oct 2021 14:15:07 +0300 Subject: [PATCH 494/950] Update profile event forwarding --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 2401b8614fa..38185af4247 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -843,7 +843,7 @@ namespace struct ProfileEventsSnapshot { UInt64 thread_id; - ProfileEvents::Counters counters; + ProfileEvents::Counters::Snapshot counters; Int64 memory_usage; time_t current_time; }; @@ -861,7 +861,7 @@ namespace auto & value_column = columns[VALUE_COLUMN_INDEX]; for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) { - UInt64 value = snapshot.counters[event].load(std::memory_order_relaxed); + UInt64 value = snapshot.counters[event]; if (value == 0) continue; From f3c9f4be6cd2a7e3d65cc9766c791111ce8159e5 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 12 Oct 2021 14:38:41 +0300 Subject: [PATCH 495/950] Update run.sh --- docker/test/stateless/run.sh | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ed721690281..2002fd2cbff 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -114,12 +114,6 @@ grep -Fa "Fatal" /var/log/clickhouse-server/clickhouse-server.log ||: pigz < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.gz & clickhouse-client -q "select * from system.query_log format TSVWithNamesAndTypes" | pigz > /test_output/query-log.tsv.gz & clickhouse-client -q "select * from system.query_thread_log format TSVWithNamesAndTypes" | pigz > /test_output/query-thread-log.tsv.gz & -clickhouse-client --allow_introspection_functions=1 -q " - WITH - arrayMap(x -> concat(demangle(addressToSymbol(x)), ':', addressToLine(x)), trace) AS trace_array, - arrayStringConcat(trace_array, '\n') AS trace_string - SELECT * EXCEPT(trace), trace_string FROM system.trace_log FORMAT TSVWithNamesAndTypes -" | pigz > /test_output/trace-log.tsv.gz & # Also export trace log in flamegraph-friendly format. for trace_type in CPU Memory Real @@ -146,6 +140,7 @@ fi tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||: tar -chf /test_output/zookeeper_log_dump.tar /var/lib/clickhouse/data/system/zookeeper_log ||: +tar -chf /test_output/trace_log_dump.tar /var/lib/clickhouse/data/system/trace_log ||: tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then From dca1f8e7f5c017ebaf263af542f2075e437412c4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 12 Oct 2021 15:38:40 +0300 Subject: [PATCH 496/950] Update README.md --- README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index d024100a27e..e12238577a7 100644 --- a/README.md +++ b/README.md @@ -1,14 +1,14 @@ [![ClickHouse — open source distributed column-oriented DBMS](https://github.com/ClickHouse/ClickHouse/raw/master/website/images/logo-400x240.png)](https://clickhouse.com) -ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real time. +ClickHouse® is an open-source column-oriented database management system that allows generating analytical data reports in real-time. ## Useful Links -* [Official website](https://clickhouse.com/) has quick high-level overview of ClickHouse on main page. -* [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query small ClickHouse cluster. +* [Official website](https://clickhouse.com/) has a quick high-level overview of ClickHouse on the main page. +* [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query a small ClickHouse cluster. * [Documentation](https://clickhouse.com/docs/en/) provides more in-depth information. * [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format. -* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-rxm3rdrk-lIUmhLC3V8WTaL0TGxsOmg) and [Telegram](https://telegram.me/clickhouse_en) allow to chat with ClickHouse users in real-time. +* [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-rxm3rdrk-lIUmhLC3V8WTaL0TGxsOmg) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. * [Blog](https://clickhouse.com/blog/en/) contains various ClickHouse-related articles, as well as announcements and reports about events. * [Code Browser](https://clickhouse.com/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation. * [Contacts](https://clickhouse.com/company/#contact) can help to get your questions answered if there are any. From 89ecbfdfe78dd5d02de2fd006b0d882db4259567 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 12 Oct 2021 15:51:19 +0300 Subject: [PATCH 497/950] Update distinctive-features.md --- docs/en/introduction/distinctive-features.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/introduction/distinctive-features.md b/docs/en/introduction/distinctive-features.md index 34ba4b89415..daf43dc0da8 100644 --- a/docs/en/introduction/distinctive-features.md +++ b/docs/en/introduction/distinctive-features.md @@ -39,9 +39,9 @@ In ClickHouse, data can reside on different shards. Each shard can be a group of ClickHouse supports a [declarative query language based on SQL](../sql-reference/index.md) that is identical to the ANSI SQL standard in [many cases](../sql-reference/ansi.md). -Supported queries include [GROUP BY](../sql-reference/statements/select/group-by.md), [ORDER BY](../sql-reference/statements/select/order-by.md), subqueries in [FROM](../sql-reference/statements/select/from.md), [JOIN](../sql-reference/statements/select/join.md) clause, [IN](../sql-reference/operators/in.md) operator, and scalar subqueries. +Supported queries include [GROUP BY](../sql-reference/statements/select/group-by.md), [ORDER BY](../sql-reference/statements/select/order-by.md), subqueries in [FROM](../sql-reference/statements/select/from.md), [JOIN](../sql-reference/statements/select/join.md) clause, [IN](../sql-reference/operators/in.md) operator, [window functions](../sql-reference/window-functions.md) and scalar subqueries. -Correlated (dependent) subqueries and window functions are not supported at the time of writing but might become available in the future. +Correlated (dependent) subqueries are not supported at the time of writing but might become available in the future. ## Vector Computation Engine {#vector-engine} From b3610134fdfa41590fbd5b4b1850bab8c47db8b8 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Tue, 12 Oct 2021 16:31:51 +0300 Subject: [PATCH 498/950] Update docs/ru/sql-reference/statements/select/prewhere.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/statements/select/prewhere.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/prewhere.md b/docs/ru/sql-reference/statements/select/prewhere.md index 84f8869b41e..4376cbeb295 100644 --- a/docs/ru/sql-reference/statements/select/prewhere.md +++ b/docs/ru/sql-reference/statements/select/prewhere.md @@ -14,7 +14,7 @@ Prewhere — это оптимизация для более эффективн В запросе может быть одновременно указаны и `PREWHERE`, и `WHERE`. В этом случае `PREWHERE` предшествует `WHERE`. -Если значение параметра [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) равно 0, эвристика по автоматическому перемещнию части выражений из `WHERE` к `PREWHERE` отключается. +Если значение параметра [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) равно 0, эвристика по автоматическому перемещению части выражений из `WHERE` к `PREWHERE` отключается. Если в запросе есть модификатор [FINAL](from.md#select-from-final), оптимизация `PREWHERE` не всегда корректна. Она действует только если включены обе настройки [optimize_move_to_prewhere](../../../operations/settings/settings.md#optimize_move_to_prewhere) и [optimize_move_to_prewhere_if_final](../../../operations/settings/settings.md#optimize_move_to_prewhere_if_final). From 0cfaf9c50861aa931b1b01e90d7e0fcc8a700472 Mon Sep 17 00:00:00 2001 From: olgarev Date: Tue, 12 Oct 2021 13:44:00 +0000 Subject: [PATCH 499/950] Unnecessary links removed --- docs/en/operations/settings/settings.md | 6 ------ docs/ru/operations/settings/settings.md | 6 ------ 2 files changed, 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0491674b701..aa70eb4f721 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3798,10 +3798,6 @@ Possible values: Default value: `1`. -**See Also** - -- [PREWHERE](../../sql-reference/statements/select/prewhere.md) clause in `SELECT` queries - ## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. @@ -3817,6 +3813,4 @@ Default value: `0`. **See Also** -- [PREWHERE](../../sql-reference/statements/select/prewhere.md) clause in `SELECT` queries -- [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier in `SELECT` queries - [optimize_move_to_prewhere](#optimize_move_to_prewhere) setting \ No newline at end of file diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 887c59c3b09..bccbbf69e39 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3587,10 +3587,6 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `1`. -**См. также** - -- секция [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах `SELECT` - ## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} Включает или отключает автоматическую оптимизацию [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах [SELECT](../../sql-reference/statements/select/index.md) с модификатором [FINAL](../../sql-reference/statements/select/from.md#select-from-final). @@ -3606,6 +3602,4 @@ SELECT * FROM positional_arguments ORDER BY 2,3; **См. также** -- секция [PREWHERE](../../sql-reference/statements/select/prewhere.md) в запросах `SELECT` -- модификатор [FINAL](../../sql-reference/statements/select/from.md#select-from-final) в запросах `SELECT` - настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) \ No newline at end of file From 9c7bef4c9d9062a48820233cfed132c2e06d5d7f Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 12 Oct 2021 17:08:47 +0300 Subject: [PATCH 500/950] Remove unused headers and handle exception 'unrecognised option' in clickhouse-local --- programs/local/LocalServer.cpp | 14 +++++--------- ...96_unknown_option_in_clickhouse_local.reference | 1 + .../02096_unknown_option_in_clickhouse_local.sh | 9 +++++++++ 3 files changed, 15 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference create mode 100755 tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 8066650006a..2035406d73a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -1,8 +1,6 @@ #include "LocalServer.h" #include -#include -#include #include #include #include @@ -10,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -21,19 +18,14 @@ #include #include #include -#include #include -#include -#include #include #include #include #include #include -#include #include #include -#include #include #include #include @@ -45,7 +37,6 @@ #include #include #include -#include #include #include @@ -722,6 +713,11 @@ int mainEntryClickHouseLocal(int argc, char ** argv) app.init(argc, argv); return app.run(); } + catch (const boost::program_options::error & e) + { + std::cerr << "Bad arguments: " << e.what() << std::endl; + return DB::ErrorCodes::BAD_ARGUMENTS; + } catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; diff --git a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference new file mode 100644 index 00000000000..96feda5dd3c --- /dev/null +++ b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference @@ -0,0 +1 @@ +Bad arguments: unrecognised option '--unknown-option' diff --git a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh new file mode 100755 index 00000000000..ee0e3f3d149 --- /dev/null +++ b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# shellcheck disable=SC2206 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} --unknown-option 2>&1 echo + From 65c6605c2ce422dc24505e66783ea2ed7c500466 Mon Sep 17 00:00:00 2001 From: Vitaly Orlov Date: Tue, 12 Oct 2021 18:00:25 +0300 Subject: [PATCH 501/950] Add JSONExtractKeys function --- src/Functions/FunctionsJSON.cpp | 1 + src/Functions/FunctionsJSON.h | 35 +++++++++++++++++++ tests/fuzz/all.dict | 1 + tests/fuzz/dictionaries/functions.dict | 1 + .../00918_json_functions.reference | 5 +++ .../0_stateless/00918_json_functions.sql | 6 ++++ 6 files changed, 49 insertions(+) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 47f485e9d6c..7fec45d1f72 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -73,6 +73,7 @@ void registerFunctionsJSON(FunctionFactory & factory) factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index b468f39af80..7407a3ec611 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -433,6 +433,7 @@ struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKey struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; +struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; template @@ -1353,4 +1354,38 @@ public: } }; +template +class JSONExtractKeysImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_unique(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + + for (auto [key, value] : object) + { + col_key.insertData(key.data(), key.size()); + } + + col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); + return true; + } +}; + } diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 73c86a13714..4a9afc348cf 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -650,6 +650,7 @@ "JSONExtractInt" "JSONExtractKeysAndValues" "JSONExtractKeysAndValuesRaw" +"JSONExtractKeys" "JSONExtractRaw" "JSONExtractString" "JSONExtractUInt" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index ba9daa72199..fb35375f284 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -31,6 +31,7 @@ "toJSONString" "JSON_VALUE" "JSONExtractKeysAndValuesRaw" +"JSONExtractKeys" "JSONExtractString" "JSONType" "JSONKey" diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 462449222b3..4c8977114e6 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -101,6 +101,11 @@ hello [('a','"hello"'),('b','[-100,200,300]')] [('a','"hello"'),('b','[-100,200,300]'),('c','{"d":[121,144]}')] [('d','[121,144]')] +--JSONExtractKeys-- +['a','b'] +[] +[] +['d'] --const/non-const mixed-- a b diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index 6504b735371..bce84df08bf 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -231,6 +231,12 @@ SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}'); SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}'); SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); +SELECT '--JSONExtractKeys--'; +SELECT JSONExtractKeys('{"a": "hello", "b": [-100, 200.0, 300]}'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); + SELECT '--const/non-const mixed--'; SELECT JSONExtractString('["a", "b", "c", "d", "e"]', idx) FROM (SELECT arrayJoin([1,2,3,4,5]) AS idx); SELECT JSONExtractString(json, 's') FROM (SELECT arrayJoin(['{"s":"u"}', '{"s":"v"}']) AS json); From f2d8cc547f52168239061a22972c805c95908457 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Oct 2021 18:09:28 +0300 Subject: [PATCH 502/950] Fix Kafka source. --- src/Storages/Kafka/KafkaSource.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 926432395e2..68fc17a97e5 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -78,8 +78,8 @@ Chunk KafkaSource::generateImpl() auto put_error_to_stream = handle_error_mode == HandleKafkaErrorMode::STREAM; - auto input_format = context->getInputFormat( - storage.getFormatName(), *buffer, non_virtual_header, max_block_size); + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); std::optional exception_message; size_t total_rows = 0; From 3540baa33c2d04788f152edb862888e66492e14a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Oct 2021 11:58:33 +0300 Subject: [PATCH 503/950] Start server under gdb in functional tests --- docker/test/stateless/run.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ed721690281..ebb72111e96 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,6 +45,23 @@ else sudo clickhouse start fi +echo " +set follow-fork-mode child +handle all noprint +handle SIGSEGV stop print +handle SIGBUS stop print +handle SIGABRT stop print +continue +thread apply all backtrace +detach +quit +" > script.gdb + +# FIXME Hung check may work incorrectly because of attached gdb +# 1. False positives are possible +# 2. We cannot attach another gdb to get stacktraces if some queries hung +gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From f845ee38d021052347b44e238c9356e4749ed611 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Oct 2021 19:15:27 +0300 Subject: [PATCH 504/950] Revert change in test --- tests/integration/test_storage_rabbitmq/test.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 85dda2fe4d3..9e2752438f8 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -463,13 +463,11 @@ def test_rabbitmq_big_message(rabbitmq_cluster): for message in messages: channel.basic_publish(exchange='big', routing_key='', body=message) - for _ in range(300): + while True: result = instance.query('SELECT count() FROM test.view') if int(result) == batch_messages * rabbitmq_messages: break - time.sleep(1) - connection.close() instance.query(''' DROP TABLE test.consumer; From 63cfc2311bb564aebfd3bb0804aca9fd20b7bfff Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 12 Oct 2021 19:38:17 +0300 Subject: [PATCH 505/950] Update distinctive-features.md --- docs/en/introduction/distinctive-features.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/distinctive-features.md b/docs/en/introduction/distinctive-features.md index daf43dc0da8..951a8a9d3e5 100644 --- a/docs/en/introduction/distinctive-features.md +++ b/docs/en/introduction/distinctive-features.md @@ -39,7 +39,7 @@ In ClickHouse, data can reside on different shards. Each shard can be a group of ClickHouse supports a [declarative query language based on SQL](../sql-reference/index.md) that is identical to the ANSI SQL standard in [many cases](../sql-reference/ansi.md). -Supported queries include [GROUP BY](../sql-reference/statements/select/group-by.md), [ORDER BY](../sql-reference/statements/select/order-by.md), subqueries in [FROM](../sql-reference/statements/select/from.md), [JOIN](../sql-reference/statements/select/join.md) clause, [IN](../sql-reference/operators/in.md) operator, [window functions](../sql-reference/window-functions.md) and scalar subqueries. +Supported queries include [GROUP BY](../sql-reference/statements/select/group-by.md), [ORDER BY](../sql-reference/statements/select/order-by.md), subqueries in [FROM](../sql-reference/statements/select/from.md), [JOIN](../sql-reference/statements/select/join.md) clause, [IN](../sql-reference/operators/in.md) operator, [window functions](../sql-reference/window-functions/index.md) and scalar subqueries. Correlated (dependent) subqueries are not supported at the time of writing but might become available in the future. From a1ab3b6606397c5748d7d478c753b96224a9c328 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 22:44:37 +0530 Subject: [PATCH 506/950] Fixed code style issue --- src/Columns/ColumnsCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 845bac79ae0..804d00d4d7c 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -270,7 +270,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; } - #elif defined(__AVX2__) + #elif defined(__AVX2__) const __m256i zero_vec = _mm256_setzero_si256(); static constexpr size_t SIMD_BYTES = 32; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From 071a5af96e6ec93a7417c5cc3df1d0f6dfc1fa7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 21:06:26 +0300 Subject: [PATCH 507/950] Revert "docker: add pandas/clickhouse_driver into test images" This reverts commit e07a6f3fc0ea0b496483287d85b50d29f5a8c330. --- docker/test/fasttest/Dockerfile | 2 +- docker/test/fuzzer/Dockerfile | 2 +- docker/test/stateless/Dockerfile | 2 +- docker/test/style/Dockerfile | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index f50c65bb9f2..798910fb952 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -67,7 +67,7 @@ RUN apt-get update \ unixodbc \ --yes --no-install-recommends -RUN pip3 install numpy scipy pandas Jinja2 pandas clickhouse_driver +RUN pip3 install numpy scipy pandas Jinja2 # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index 13353bc2960..6444e745c47 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -27,7 +27,7 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -RUN pip3 install Jinja2 pandas clickhouse_driver +RUN pip3 install Jinja2 COPY * / diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index a5733d11dd2..7de8c061673 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -34,7 +34,7 @@ RUN apt-get update -y \ postgresql-client \ sqlite3 -RUN pip3 install numpy scipy pandas Jinja2 clickhouse_driver +RUN pip3 install numpy scipy pandas Jinja2 RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile index 64cc0c9c7b7..33cdb9db57a 100644 --- a/docker/test/style/Dockerfile +++ b/docker/test/style/Dockerfile @@ -10,7 +10,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ python3-pip \ pylint \ yamllint \ - && pip3 install codespell pandas clickhouse_driver + && pip3 install codespell COPY run.sh / COPY process_style_check_result.py / From e2e62ce2735a70bf77fdf3f2277728a338ab034f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 21:06:26 +0300 Subject: [PATCH 508/950] clickhouse-test: replace clickhouse-driver with http interface (via http.client) Cons of clickhouse-driver: - it is one more extra dependency - it does not have correct timeouts (only for socket operations, and this is not the same, so we need to set timeout by ourself) - it is one more thing which can break (@alesapin) --- tests/clickhouse-test | 176 +++++++++++++++++++----------------------- 1 file changed, 81 insertions(+), 95 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 62860a36fc7..d73c73a3650 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -13,6 +13,10 @@ import re import copy import traceback import math +# Not requests, to avoid requiring extra dependency. +import http.client +import urllib.parse +import json from argparse import ArgumentParser from typing import Tuple, Union, Optional, Dict, Set, List @@ -34,9 +38,6 @@ import multiprocessing import socket from contextlib import closing -import clickhouse_driver -import pandas - USE_JINJA = True try: import jinja2 @@ -51,53 +52,60 @@ MESSAGES_TO_RETRY = [ "DB::Exception: Cannot enqueue query", "is executing longer than distributed_ddl_task_timeout" # FIXME ] -error_codes = clickhouse_driver.errors.ErrorCodes -error_codes.NOT_A_LEADER = 529 -ERROR_CODES_TO_RETRY = [ - error_codes.ALL_CONNECTION_TRIES_FAILED, - error_codes.DATABASE_NOT_EMPTY, - error_codes.NOT_A_LEADER, - error_codes.UNFINISHED, -] MAX_RETRIES = 3 TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect'] -class Client(clickhouse_driver.Client): - # return first column of the first row - def execute_one(self, *args, **kwargs): - return super().execute(*args, **kwargs)[0][0] +class HTTPError(Exception): + def __init__(self, message=None, code=None): + self.message = message + self.code = code + super().__init__(message) - # return pandas.DataFrame - def execute_pandas(self, *args, **kwargs): - data = super().execute(*args, **kwargs, with_column_types=True) - return Client.__combine(data) + def __str__(self): + return 'Code: {}. {}'.format(self.code, self.message) - @staticmethod - def __combine(data): - cols = data[1] - rows = data[0] - header = [ i[0] for i in cols ] - data = pandas.DataFrame(data=rows, columns=header) - return data +# Helpers to execute queries via HTTP interface. +def clickhouse_execute_http(base_args, query, timeout=30, settings=None, default_format=None): + client = http.client.HTTPConnection( + host=base_args.tcp_host, + port=base_args.http_port, + timeout=timeout) -# Helpers -def make_clickhouse_client(base_args): - return Client(host=base_args.tcp_host, port=base_args.tcp_port, - # hung check in stress tests may remove the database, - # hence we should use 'system'. - database='system', - settings=get_additional_client_options_dict(base_args)) + timeout = int(timeout) + params = { + 'query': query, -def clickhouse_execute_one(base_args, *args, **kwargs): - return make_clickhouse_client(base_args).execute_one(*args, **kwargs) + 'connect_timeout': timeout, + 'receive_timeout': timeout, + 'send_timeout': timeout, -def clickhouse_execute(base_args, *args, **kwargs): - return make_clickhouse_client(base_args).execute(*args, **kwargs) + 'http_connection_timeout': timeout, + 'http_receive_timeout': timeout, + 'http_send_timeout': timeout, + } + if settings is not None: + params.update(settings) + if default_format is not None: + params['default_format'] = default_format -def clickhouse_execute_pandas(base_args, *args, **kwargs): - return make_clickhouse_client(base_args).execute_pandas(*args, **kwargs) + client.request('POST', '/?' + urllib.parse.urlencode(params)) + res = client.getresponse() + data = res.read() + if res.status != 200: + raise HTTPError(data.decode(), res.status) + + return data + +def clickhouse_execute(base_args, query, timeout=30, settings=None): + return clickhouse_execute_http(base_args, query, timeout, settings).strip() + +def clickhouse_execute_json(base_args, query, timeout=30, settings=None): + data = clickhouse_execute_http(base_args, query, timeout, settings, 'JSONEachRow') + if not data: + return None + return json.loads(data) class Terminated(KeyboardInterrupt): @@ -144,12 +152,12 @@ def get_db_engine(args, database_name): def get_zookeeper_session_uptime(args): try: if args.replicated_database: - return int(clickhouse_execute_one(args, """ + return int(clickhouse_execute(args, """ SELECT min(materialize(zookeeperSessionUptime())) FROM clusterAllReplicas('test_cluster_database_replicated', system.one) """)) else: - return int(clickhouse_execute_one(args, 'SELECT zookeeperSessionUptime()')) + return int(clickhouse_execute(args, 'SELECT zookeeperSessionUptime()')) except: return None @@ -163,30 +171,16 @@ def need_retry(args, stdout, stderr, total_time): return True return any(msg in stdout for msg in MESSAGES_TO_RETRY) or any(msg in stderr for msg in MESSAGES_TO_RETRY) -def need_retry_error(args, error, total_time): - # Sometimes we may get unexpected exception like "Replica is readonly" or "Shutdown is called for table" - # instead of "Session expired" or "Connection loss" - # Retry if session was expired during test execution - session_uptime = get_zookeeper_session_uptime(args) - if session_uptime is not None and session_uptime < math.ceil(total_time): - return True - if isinstance(error, clickhouse_driver.errors.Error): - if error.code in ERROR_CODES_TO_RETRY: - return True - if any(msg in error.message for msg in MESSAGES_TO_RETRY): - return True - return False - def get_processlist(args): if args.replicated_database: - return clickhouse_execute_pandas(args, """ + return clickhouse_execute_json(args, """ SELECT materialize((hostName(), tcpPort())) as host, * FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) WHERE query NOT LIKE '%system.processes%' """) else: - return clickhouse_execute_pandas(args, 'SHOW PROCESSLIST') + return clickhouse_execute_json(args, 'SHOW PROCESSLIST') # collect server stacktraces using gdb @@ -374,7 +368,7 @@ class TestCase: try: clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename}) - except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError): + except TimeoutError: total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() return None, "", f"Timeout creating database {database} before test", total_time @@ -593,11 +587,8 @@ class TestCase: if need_drop_database: seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) try: - client = make_clickhouse_client(args) - client.connection.force_connect() - with client.connection.timeout_setter(seconds_left): - client.execute("DROP DATABASE " + database) - except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError): + clickhouse_execute(args, "DROP DATABASE " + database, timeout=seconds_left) + except TimeoutError: total_time = (datetime.now() - start_time).total_seconds() return None, "", f"Timeout dropping database {database} after test", total_time shutil.rmtree(args.test_tmp_dir) @@ -803,7 +794,7 @@ class TestSuite: @staticmethod def readTestSuite(args, suite_dir_name: str): def is_data_present(): - return int(clickhouse_execute_one(args, 'EXISTS TABLE test.hits')) + return int(clickhouse_execute(args, 'EXISTS TABLE test.hits')) base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) @@ -976,7 +967,7 @@ def check_server_started(args): print(" OK") sys.stdout.flush() return True - except (ConnectionRefusedError, ConnectionResetError, clickhouse_driver.errors.NetworkError): + except (ConnectionRefusedError, ConnectionResetError): print('.', end='') sys.stdout.flush() retry_count -= 1 @@ -1003,31 +994,31 @@ class BuildFlags(): def collect_build_flags(args): result = [] - value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") - if '-fsanitize=thread' in value: + value = clickhouse_execute(args, "SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") + if b'-fsanitize=thread' in value: result.append(BuildFlags.THREAD) - elif '-fsanitize=address' in value: + elif b'-fsanitize=address' in value: result.append(BuildFlags.ADDRESS) - elif '-fsanitize=undefined' in value: + elif b'-fsanitize=undefined' in value: result.append(BuildFlags.UNDEFINED) - elif '-fsanitize=memory' in value: + elif b'-fsanitize=memory' in value: result.append(BuildFlags.MEMORY) - value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") - if 'Debug' in value: + value = clickhouse_execute(args, "SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") + if b'Debug' in value: result.append(BuildFlags.DEBUG) - elif 'RelWithDebInfo' in value or 'Release' in value: + elif b'RelWithDebInfo' in value or b'Release' in value: result.append(BuildFlags.RELEASE) - value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") - if value in ('ON', '1'): + value = clickhouse_execute(args, "SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") + if value in (b'ON', b'1'): result.append(BuildFlags.UNBUNDLED) - value = clickhouse_execute_one(args, "SELECT value FROM system.settings WHERE name = 'default_database_engine'") - if value == 'Ordinary': + value = clickhouse_execute(args, "SELECT value FROM system.settings WHERE name = 'default_database_engine'") + if value == b'Ordinary': result.append(BuildFlags.ORDINARY_DATABASE) - value = int(clickhouse_execute_one(args, "SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")) + value = int(clickhouse_execute(args, "SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")) if value == 0: result.append(BuildFlags.POLYMORPHIC_PARTS) @@ -1173,9 +1164,9 @@ def main(args): start_time = datetime.now() try: clickhouse_execute(args, "CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name)) - except Exception as e: + except HTTPError as e: total_time = (datetime.now() - start_time).total_seconds() - if not need_retry_error(args, e, total_time): + if not need_retry(args, e.message, e.message, total_time): break create_database_retries += 1 @@ -1204,13 +1195,13 @@ def main(args): # Some queries may execute in background for some time after test was finished. This is normal. for _ in range(1, 60): processlist = get_processlist(args) - if processlist.empty: + if not processlist: break sleep(1) - if not processlist.empty: + if processlist: print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) - print(processlist) + print(json.dumps(processlist, indent=4)) print_stacktraces() exit_code.value = 1 @@ -1262,14 +1253,6 @@ def get_additional_client_options_url(args): return '&'.join(args.client_option) return '' -def get_additional_client_options_dict(args): - settings = {} - if args.client_option: - for key, value in map(lambda x: x.split('='), args.client_option): - settings[key] = value - return settings - - if __name__ == '__main__': stop_time = None exit_code = multiprocessing.Value("i", 0) @@ -1401,6 +1384,13 @@ if __name__ == '__main__': else: args.tcp_port = 9000 + http_port = os.getenv("CLICKHOUSE_PORT_HTTP") + if http_port is not None: + args.http_port = int(http_port) + args.client += f" --port={http_port}" + else: + args.http_port = 8123 + client_database = os.getenv("CLICKHOUSE_DATABASE") if client_database is not None: args.client += f' --database={client_database}' @@ -1434,8 +1424,4 @@ if __name__ == '__main__': if args.jobs is None: args.jobs = multiprocessing.cpu_count() - # configure pandas to make it more like Vertical format - pandas.options.display.max_columns = None - pandas.options.display.width = None - main(args) From 50b95bd89d9b6d6b563d8d084379a7f7d5341ed8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 21:06:26 +0300 Subject: [PATCH 509/950] clickhouse-test: passthrough log_comment for DROP DATABASE too --- tests/clickhouse-test | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index d73c73a3650..2e2b34ac68c 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -352,6 +352,7 @@ class TestCase: testcase_args.testcase_start_time = datetime.now() testcase_basename = os.path.basename(case_file) testcase_args.testcase_client = f"{testcase_args.client} --log_comment='{testcase_basename}'" + testcase_args.testcase_basename = testcase_basename if testcase_args.database: database = testcase_args.database @@ -367,7 +368,9 @@ class TestCase: database = 'test_{suffix}'.format(suffix=random_str()) try: - clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename}) + clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={ + 'log_comment': testcase_args.testcase_basename, + }) except TimeoutError: total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() return None, "", f"Timeout creating database {database} before test", total_time @@ -587,7 +590,9 @@ class TestCase: if need_drop_database: seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20) try: - clickhouse_execute(args, "DROP DATABASE " + database, timeout=seconds_left) + clickhouse_execute(args, "DROP DATABASE " + database, timeout=seconds_left, settings={ + 'log_comment': args.testcase_basename, + }) except TimeoutError: total_time = (datetime.now() - start_time).total_seconds() return None, "", f"Timeout dropping database {database} after test", total_time From fe90c979b59cddc135eff510a9dc880b6bb0ef54 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 20:13:46 +0300 Subject: [PATCH 510/950] clickhouse-test: fix redirect to stderr Before #29856 `CREATE DATABASE` overwrites it. Reported-by: @amosbird --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2e2b34ac68c..b28ea2a49ac 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -567,7 +567,7 @@ class TestCase: # >> append to stderr (but not stdout since it is not used there), # because there are also output of per test database creation if not args.database: - pattern = '{test} > {stdout} 2>> {stderr}' + pattern = '{test} > {stdout} 2> {stderr}' else: pattern = '{test} > {stdout} 2> {stderr}' From 4d5b793f2b6d98d34cc90e7038e89b7d187c6f58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 22:22:49 +0300 Subject: [PATCH 511/950] Update submodule --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 311cfd49896..79358a3106a 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 311cfd498966d4f77742703d605d9c2e7b4cc6a8 +Subproject commit 79358a3106aab6af464430ed67c7efafebf5cd6f From 8d6126fd911c27d64e431bcfa74622346b4225ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 22:25:45 +0300 Subject: [PATCH 512/950] Fix build --- cmake/target.cmake | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cmake/target.cmake b/cmake/target.cmake index 08a27160985..c1a34d0df13 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -34,6 +34,8 @@ if (CMAKE_CROSSCOMPILING) endif () elseif (OS_FREEBSD) # FIXME: broken dependencies + set (ENABLE_PARQUET OFF CACHE INTERNAL "") + set (ENABLE_ORC OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") else () message (FATAL_ERROR "Trying to cross-compile to unsupported system: ${CMAKE_SYSTEM_NAME}!") From 8dc8674298f35bdc1bffe201256e2cf1a7a4e15c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 22:27:54 +0300 Subject: [PATCH 513/950] clickhouse-test: process --client-option --- tests/clickhouse-test | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b28ea2a49ac..024b4c2ae0f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -90,7 +90,7 @@ def clickhouse_execute_http(base_args, query, timeout=30, settings=None, default if default_format is not None: params['default_format'] = default_format - client.request('POST', '/?' + urllib.parse.urlencode(params)) + client.request('POST', '/?' + base_args.client_options_query_str + urllib.parse.urlencode(params)) res = client.getresponse() data = res.read() if res.status != 200: @@ -1418,7 +1418,11 @@ if __name__ == '__main__': else: os.environ['CLICKHOUSE_URL_PARAMS'] = '' - os.environ['CLICKHOUSE_URL_PARAMS'] += get_additional_client_options_url(args) + client_options_query_str = get_additional_client_options_url(args) + args.client_options_query_str = client_options_query_str + '&' + os.environ['CLICKHOUSE_URL_PARAMS'] += client_options_query_str + else: + args.client_options_query_str = '' if args.extract_from_config is None: if os.access(args.binary + '-extract-from-config', os.X_OK): From 427c428a27f418c38b4171e8171a888d9f22e31b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 22:30:34 +0300 Subject: [PATCH 514/950] clickhouse-test: process some options regardless --client --- tests/clickhouse-test | 52 +++++++++++++++++++++---------------------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 024b4c2ae0f..a70b8795142 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1372,36 +1372,36 @@ if __name__ == '__main__': parser.print_help() sys.exit(1) - if args.configclient: - args.client += ' --config-file=' + args.configclient + if args.configclient: + args.client += ' --config-file=' + args.configclient - tcp_host = os.getenv("CLICKHOUSE_HOST") - if tcp_host is not None: - args.tcp_host = tcp_host - args.client += f' --host={tcp_host}' - else: - args.tcp_host = 'localhost' + tcp_host = os.getenv("CLICKHOUSE_HOST") + if tcp_host is not None: + args.tcp_host = tcp_host + args.client += f' --host={tcp_host}' + else: + args.tcp_host = 'localhost' - tcp_port = os.getenv("CLICKHOUSE_PORT_TCP") - if tcp_port is not None: - args.tcp_port = int(tcp_port) - args.client += f" --port={tcp_port}" - else: - args.tcp_port = 9000 + tcp_port = os.getenv("CLICKHOUSE_PORT_TCP") + if tcp_port is not None: + args.tcp_port = int(tcp_port) + args.client += f" --port={tcp_port}" + else: + args.tcp_port = 9000 - http_port = os.getenv("CLICKHOUSE_PORT_HTTP") - if http_port is not None: - args.http_port = int(http_port) - args.client += f" --port={http_port}" - else: - args.http_port = 8123 + http_port = os.getenv("CLICKHOUSE_PORT_HTTP") + if http_port is not None: + args.http_port = int(http_port) + args.client += f" --port={http_port}" + else: + args.http_port = 8123 - client_database = os.getenv("CLICKHOUSE_DATABASE") - if client_database is not None: - args.client += f' --database={client_database}' - args.client_database = client_database - else: - args.client_database = 'default' + client_database = os.getenv("CLICKHOUSE_DATABASE") + if client_database is not None: + args.client += f' --database={client_database}' + args.client_database = client_database + else: + args.client_database = 'default' if args.client_option: # Set options for client From 40d210367fd23846f58e431acd5f1eb16d70e1b2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 22:33:41 +0300 Subject: [PATCH 515/950] clickhouse-test: fix catching of timeouts --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a70b8795142..aedf5f5be67 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -371,7 +371,7 @@ class TestCase: clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={ 'log_comment': testcase_args.testcase_basename, }) - except TimeoutError: + except socket.timeout: total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() return None, "", f"Timeout creating database {database} before test", total_time @@ -593,7 +593,7 @@ class TestCase: clickhouse_execute(args, "DROP DATABASE " + database, timeout=seconds_left, settings={ 'log_comment': args.testcase_basename, }) - except TimeoutError: + except socket.timeout: total_time = (datetime.now() - start_time).total_seconds() return None, "", f"Timeout dropping database {database} after test", total_time shutil.rmtree(args.test_tmp_dir) From 8d7798fa2e220d989106efdec375c81e2ace8fc3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 22:39:43 +0300 Subject: [PATCH 516/950] clickhouse-test: do not guard CREATE DATABASE with try/catch Since it will not configure testcase args and fail eventually, and later we have a try/catch anyway, this should be enough. --- tests/clickhouse-test | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index aedf5f5be67..133c6f4e4e8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -367,13 +367,9 @@ class TestCase: database = 'test_{suffix}'.format(suffix=random_str()) - try: - clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={ - 'log_comment': testcase_args.testcase_basename, - }) - except socket.timeout: - total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds() - return None, "", f"Timeout creating database {database} before test", total_time + clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={ + 'log_comment': testcase_args.testcase_basename, + }) os.environ["CLICKHOUSE_DATABASE"] = database # Set temporary directory to match the randomly generated database, From 65b63a67da556ca671ad7c8c1a3c28a305097b1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 23:03:20 +0300 Subject: [PATCH 517/950] Add a script for convenient install on multiple OS --- docs/_includes/install/universal.sh | 59 +++++++++++++++++++++++++++++ 1 file changed, 59 insertions(+) create mode 100755 docs/_includes/install/universal.sh diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh new file mode 100755 index 00000000000..b5f833b19a6 --- /dev/null +++ b/docs/_includes/install/universal.sh @@ -0,0 +1,59 @@ +#!/bin/sh -e + +OS=$(uname -s) +ARCH=$(uname -m) + +DIR= + +if [ "${OS}" = "Linux" ] +then + if [ "${ARCH}" = "x86_64" ] + then + DIR="amd64" + elif [ "${ARCH}" = "aarch64" ] + then + DIR="aarch64" + elif [ "${ARCH}" = "powerpc64le" ] + then + DIR="powerpc64le" + fi +elif [ "${OS}" = "FreeBSD" ] +then + if [ "${ARCH}" = "x86_64" ] + then + DIR="freebsd" + elif [ "${ARCH}" = "aarch64" ] + then + #DIR="freebsd-aarch64" + elif [ "${ARCH}" = "powerpc64le" ] + then + #DIR="freebsd-powerpc64le" + fi +elif [ "${OS}" = "Darwin" ] +then + if [ "${ARCH}" = "x86_64" ] + then + DIR="macos" + elif [ "${ARCH}" = "aarch64" ] + then + DIR="macos-aarch64" + fi +fi + +if [ -z "${DIR}" ] +then + echo "The '${OS}' operating system with the '${ARCH}' architecture is not supported." + exit 1 +fi + +URL="https://builds.clickhouse.com/master/${DIR}/clickhouse" +echo "Will download ${URL}" +curl -O "${URL}" && chmod a+x clickhouse && +echo "Successfully downloaded the ClickHouse binary, you can run it as: + ./clickhouse" + +if [ "${OS}" = "Linux" ] +then + echo "You can also install it: + ./clickhouse install" +fi From ac403b1df259df7084c89d57ea77bfad4c914783 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Oct 2021 23:09:26 +0300 Subject: [PATCH 518/950] Publish the install script --- docs/_includes/install/universal.sh | 2 +- docs/tools/website.py | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index b5f833b19a6..db1072f149f 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -55,5 +55,5 @@ echo "Successfully downloaded the ClickHouse binary, you can run it as: if [ "${OS}" = "Linux" ] then echo "You can also install it: - ./clickhouse install" + sudo ./clickhouse install" fi diff --git a/docs/tools/website.py b/docs/tools/website.py index 5e4f48e3441..54804ae3f36 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -156,6 +156,11 @@ def build_website(args): os.path.join(args.src_dir, 'utils', 'list-versions', 'version_date.tsv'), os.path.join(args.output_dir, 'data', 'version_date.tsv')) + # This file can be requested to install ClickHouse. + shutil.copy2( + os.path.join(args.src_dir, 'docs', '_includes', 'install', 'universal.sh'), + os.path.join(args.output_dir, 'data', 'install.sh')) + for root, _, filenames in os.walk(args.output_dir): for filename in filenames: if filename == 'main.html': @@ -218,7 +223,7 @@ def minify_file(path, css_digest, js_digest): # TODO: restore cssmin # elif path.endswith('.css'): # content = cssmin.cssmin(content) -# TODO: restore jsmin +# TODO: restore jsmin # elif path.endswith('.js'): # content = jsmin.jsmin(content) with open(path, 'wb') as f: From f7aaa3d282df5988bd4a5a87f460d08513547420 Mon Sep 17 00:00:00 2001 From: Alexey Date: Tue, 12 Oct 2021 20:15:49 +0000 Subject: [PATCH 519/950] Unlink before creating a link --- docs/tools/cmake_in_clickhouse_generator.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/tools/cmake_in_clickhouse_generator.py b/docs/tools/cmake_in_clickhouse_generator.py index 8b440823df3..e66915d4a39 100644 --- a/docs/tools/cmake_in_clickhouse_generator.py +++ b/docs/tools/cmake_in_clickhouse_generator.py @@ -160,7 +160,10 @@ def generate_cmake_flags_files() -> None: "docs/ru/development/cmake-in-clickhouse.md"] for lang in other_languages: - os.symlink(output_file_name, os.path.join(root_path, lang)) + other_file_name = os.path.join(root_path, lang) + if os.path.exists(other_file_name): + os.unlink(other_file_name) + os.symlink(output_file_name, other_file_name) if __name__ == '__main__': generate_cmake_flags_files() From f1bbc7f9b633f5f3948358e574dad4d71c3f7199 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 12 Oct 2021 23:17:15 +0300 Subject: [PATCH 520/950] Fix hardware utilization info printing in client --- docs/en/interfaces/cli.md | 1 + src/Client/ClientBase.cpp | 5 +++- src/Common/ProgressIndication.cpp | 41 +++++++++++++++++-------------- src/Common/ProgressIndication.h | 2 ++ 4 files changed, 30 insertions(+), 19 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 70b7d59b037..c4305d229cf 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -127,6 +127,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--secure` – If specified, will connect to server over secure connection. - `--history_file` — Path to a file containing command history. - `--param_` — Value for a [query with parameters](#cli-queries-with-parameters). +- `--hardware-utilization` — Print hardware utilization information in progress bar. Since version 20.5, `clickhouse-client` has automatic syntax highlighting (always enabled). diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index baf082a3541..c93645a1f8a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -668,7 +668,7 @@ void ClientBase::onEndOfStream() void ClientBase::onProfileEvents(Block & block) { const auto rows = block.rows(); - if (rows == 0) + if (rows == 0 || !progress_indication.print_hardware_utilization) return; const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & names = typeid_cast(*block.getByName("name").column); @@ -1560,6 +1560,7 @@ void ClientBase::init(int argc, char ** argv) ("ignore-error", "do not stop processing in multiquery mode") ("stacktrace", "print stack traces of exceptions") + ("hardware-utilization", "print hardware utilization information in progress bar") ; addAndCheckOptions(options_description, options, common_arguments); @@ -1626,6 +1627,8 @@ void ClientBase::init(int argc, char ** argv) config().setBool("verbose", true); if (options.count("log-level")) Poco::Logger::root().setLevel(options["log-level"].as()); + if (options.count("hardware-utilization")) + progress_indication.print_hardware_utilization = true; query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 0fe40b306cb..4510952cc71 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "IO/WriteBufferFromString.h" #include @@ -189,6 +190,26 @@ void ProgressIndication::writeProgress() written_progress_chars = message.count() - prefix_size - (strlen(indicator) - 2); /// Don't count invisible output (escape sequences). + // If approximate cores number is known, display it. + auto cores_number = getApproximateCoresNumber(); + std::string profiling_msg; + if (cores_number != 0 && print_hardware_utilization) + { + WriteBufferFromOwnString profiling_msg_builder; + // Calculated cores number may be not accurate + // so it's better to print min(threads, cores). + UInt64 threads_number = getUsedThreadsCount(); + profiling_msg_builder << " Running " << threads_number << " threads on " + << std::min(cores_number, threads_number) << " cores"; + + auto memory_usage = getMemoryUsage(); + if (memory_usage != 0) + profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; + else + profiling_msg_builder << "."; + profiling_msg = profiling_msg_builder.str(); + } + /// If the approximate number of rows to process is known, we can display a progress bar and percentage. if (progress.total_rows_to_read || progress.total_raw_bytes_to_read) { @@ -215,7 +236,7 @@ void ProgressIndication::writeProgress() if (show_progress_bar) { - ssize_t width_of_progress_bar = static_cast(terminal_width) - written_progress_chars - strlen(" 99%"); + ssize_t width_of_progress_bar = static_cast(terminal_width) - written_progress_chars - strlen(" 99%") - profiling_msg.length(); if (width_of_progress_bar > 0) { std::string bar @@ -231,23 +252,7 @@ void ProgressIndication::writeProgress() message << ' ' << (99 * current_count / max_count) << '%'; } - // If approximate cores number is known, display it. - auto cores_number = getApproximateCoresNumber(); - if (cores_number != 0) - { - // Calculated cores number may be not accurate - // so it's better to print min(threads, cores). - UInt64 threads_number = getUsedThreadsCount(); - message << " Running " << threads_number << " threads on " - << std::min(cores_number, threads_number) << " cores"; - - auto memory_usage = getMemoryUsage(); - if (memory_usage != 0) - message << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; - else - message << "."; - } - + message << profiling_msg; message << CLEAR_TO_END_OF_LINE; ++increment; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 3d9bbc7f3ff..9b1b2b0b145 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -60,6 +60,8 @@ public: void updateThreadEventData(HostToThreadTimesMap & new_thread_data); + bool print_hardware_utilization = false; + private: size_t getUsedThreadsCount() const; From a30573fc0a8ceb3b9deae5d667f886f819bdac1f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 30 Sep 2021 18:44:30 +0300 Subject: [PATCH 521/950] Add FAIL message to test_results.tsv --- .../util/process_functional_tests_result.py | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index e60424ad4d1..ae8d49836d8 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -49,19 +49,24 @@ def process_test_log(log_path): total += 1 if TIMEOUT_SIGN in line: failed += 1 - test_results.append((test_name, "Timeout", test_time)) + test_results.append((test_name, "Timeout", test_time, [])) elif FAIL_SIGN in line: failed += 1 - test_results.append((test_name, "FAIL", test_time)) + test_results.append((test_name, "FAIL", test_time, [])) elif UNKNOWN_SIGN in line: unknown += 1 - test_results.append((test_name, "FAIL", test_time)) + test_results.append((test_name, "FAIL", test_time, [])) elif SKIPPED_SIGN in line: skipped += 1 - test_results.append((test_name, "SKIPPED", test_time)) + test_results.append((test_name, "SKIPPED", test_time, [])) else: success += int(OK_SIGN in line) - test_results.append((test_name, "OK", test_time)) + test_results.append((test_name, "OK", test_time, [])) + elif len(test_results) > 0 and test_results[-1][1] == "FAIL": + test_results[-1][3].append(line) + + test_results = [(test[0], test[1], test[2], ''.join(test[3])) for test in test_results] + return total, skipped, unknown, failed, success, hung, task_timeout, retries, test_results def process_result(result_path): @@ -89,14 +94,14 @@ def process_result(result_path): if hung: description = "Some queries hung, " state = "failure" - test_results.append(("Some queries hung", "FAIL", "0")) + test_results.append(("Some queries hung", "FAIL", "0", "")) elif task_timeout: description = "Timeout, " state = "failure" - test_results.append(("Timeout", "FAIL", "0")) + test_results.append(("Timeout", "FAIL", "0", "")) elif retries: description = "Some tests restarted, " - test_results.append(("Some tests restarted", "SKIPPED", "0")) + test_results.append(("Some tests restarted", "SKIPPED", "0", "")) else: description = "" From 05073910104801c2a40e3fba33932422abc15ae3 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 6 Oct 2021 23:07:30 +0300 Subject: [PATCH 522/950] Do not stop CI tests processing if clickhouse-test return 1 --- docker/test/fasttest/run.sh | 2 ++ docker/test/stateful/run.sh | 2 ++ docker/test/stateless/run.sh | 2 ++ 3 files changed, 6 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c8a3ad7c998..f4b99603554 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -262,11 +262,13 @@ function run_tests start_server + set +e time clickhouse-test --hung-check -j 8 --order=random \ --fast-tests-only --no-long --testname --shard --zookeeper \ -- "$FASTTEST_FOCUS" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee "$FASTTEST_OUTPUT/test_result.txt" + set -e } case "$stage" in diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index dd5984fd7b5..69b435857d9 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -108,8 +108,10 @@ function run_tests() ADDITIONAL_OPTIONS+=('--replicated-database') fi + set +e clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt + set -e } export -f run_tests diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ed721690281..97e1ea955b1 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -96,10 +96,12 @@ function run_tests() ADDITIONAL_OPTIONS+=('8') fi + set +e clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt + set -e } export -f run_tests From fa9cdd5c5fba6719cf21f1b31d805ff9233a2b93 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 7 Oct 2021 12:49:41 +0300 Subject: [PATCH 523/950] Use original whitespaces in test_results.tsv --- docker/test/util/process_functional_tests_result.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index ae8d49836d8..82df170686d 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -28,6 +28,7 @@ def process_test_log(log_path): test_results = [] with open(log_path, 'r') as test_file: for line in test_file: + original_line = line line = line.strip() if any(s in line for s in NO_TASK_TIMEOUT_SIGNS): task_timeout = False @@ -63,7 +64,7 @@ def process_test_log(log_path): success += int(OK_SIGN in line) test_results.append((test_name, "OK", test_time, [])) elif len(test_results) > 0 and test_results[-1][1] == "FAIL": - test_results[-1][3].append(line) + test_results[-1][3].append(original_line) test_results = [(test[0], test[1], test[2], ''.join(test[3])) for test in test_results] From 38cfc347447102d3b32962144127b40856823e9c Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:44:23 +0300 Subject: [PATCH 524/950] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 7a157dbe946..cf7160c15c8 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -71,7 +71,7 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression. ## encryption {#server-settings-encryption} -It configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Key (or keys) should be written in environment variables or set in the configuration file. +Configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Key (or keys) should be written in environment variables or set in the configuration file. Keys can be hex or string with the length equal to 16 bytes. From 35c007c82cadeb38230df248678bd3e9c29c8735 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:44:42 +0300 Subject: [PATCH 525/950] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index cf7160c15c8..19567ec29fb 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -73,7 +73,7 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression. Configures a command to obtain a key to be used by [encryption codecs](../../sql-reference/statements/create/table.md#create-query-encryption-codecs). Key (or keys) should be written in environment variables or set in the configuration file. -Keys can be hex or string with the length equal to 16 bytes. +Keys can be hex or string with a length equal to 16 bytes. **Example** From 3f81281e4c2212e8ced56d9d3daa61f2973dc8c6 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:44:58 +0300 Subject: [PATCH 526/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 772f07fe8ec..6a527aa21ca 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -92,7 +92,7 @@ SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); ## toInt(8\|16\|32\|64\|128\|256)OrDefault {#toint8163264128256orDefault} -Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64\|128\|256). Если не удалось - возвращает значение по умолчанию. +Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64\|128\|256). Если не удалось — возвращает значение по умолчанию. **Пример** From 007ac0f61c588a68c3b91fe8f8e266749a439c34 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:45:16 +0300 Subject: [PATCH 527/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 6a527aa21ca..6531b79464d 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -110,7 +110,6 @@ SELECT toInt64OrDefault('123123', cast('-1' as Int64)), toInt8OrDefault('123qwe1 └─────────────────────────────────────────────────┴──────────────────────────────────────────────────┘ ``` - ## toUInt(8\|16\|32\|64\|256) {#touint8163264} Преобраует входное значение к типу [UInt](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает: From 3dcc00077c9ec96d786698575100aaf962217a5c Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:46:25 +0300 Subject: [PATCH 528/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 6531b79464d..119c9ea587f 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -856,7 +856,7 @@ SELECT ## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_null} -Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если указано. +Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если оно указано. **Синтаксис** From a24a8f4bdc6af9cea08e69269b2f31c6e87bec51 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:48:19 +0300 Subject: [PATCH 529/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 119c9ea587f..38911a2e88c 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -365,7 +365,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); Преобразует входную строку в значение с типом данных [Decimal(P,S)](../../sql-reference/data-types/decimal.md). Семейство функций включает в себя: -- `toDecimal32OrDefault(expr, S)` — Возвращает значение типа `Decimal32(S)` data type. +- `toDecimal32OrDefault(expr, S)` — возвращает значение типа `Decimal32(S)`. - `toDecimal64OrDefault(expr, S)` — Возвращает значение типа `Decimal64(S)` data type. - `toDecimal128OrDefault(expr, S)` — Возвращает значение типа `Decimal128(S)` data type. - `toDecimal256OrDefault(expr, S)` — Возвращает значение типа `Decimal256(S)` data type. From 835eb86b1c23ceeb6a6c6fba6c2348e23c123bd9 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:48:39 +0300 Subject: [PATCH 530/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 38911a2e88c..4ae6a32167d 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -853,7 +853,6 @@ SELECT ``` - ## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_null} Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если оно указано. From 442fe2d979a4a021506d30b265d905a947043c4d Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:49:31 +0300 Subject: [PATCH 531/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 4ae6a32167d..995e21c297f 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -853,7 +853,7 @@ SELECT ``` -## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_null} +## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_default} Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если оно указано. From 4a64265db8e0332e554591fe6cf2430f4f4c682b Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:49:39 +0300 Subject: [PATCH 532/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 995e21c297f..fbe4bfd8d30 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -854,7 +854,6 @@ SELECT ## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_default} - Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если оно указано. **Синтаксис** From 19217af59dc73a1ff019fc6fc37893127af75222 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:50:07 +0300 Subject: [PATCH 533/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index fbe4bfd8d30..1c1a3717751 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -361,7 +361,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); ``` -## toDecimal(32\|64\|128\|256)OrDefault {#todecimal3264128256ornull} +## toDecimal(32\|64\|128\|256)OrDefault {#todecimal3264128256ordefault} Преобразует входную строку в значение с типом данных [Decimal(P,S)](../../sql-reference/data-types/decimal.md). Семейство функций включает в себя: From 0c645aaeec2328179669d270cf2f736c227986be Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:50:14 +0300 Subject: [PATCH 534/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 1c1a3717751..f85531f77f3 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -278,7 +278,7 @@ SELECT toDate32OrNull('1955-01-01'), toDate32OrNull(''); └──────────────────────────────┴────────────────────┘ ``` -## toDate32OrDefault {#todate32-or-null} +## toDate32OrDefault {#todate32-or-default} То же самое, что и [toDate32](#todate32), но возвращает значение по умолчанию, если получен недопустимый аргумент. From 601050ea38bf9640800308b2f5f55c32e476e9a8 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:50:25 +0300 Subject: [PATCH 535/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index f85531f77f3..da147ff98ca 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -866,7 +866,7 @@ accurateCastOrDefault(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. -- `default_value` - значение по умолчанию возвращаемого типа данных. +- `default_value` — значение по умолчанию возвращаемого типа данных. **Возвращаемое значение** From e120de60d4aa009c492d163087445608832c5724 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:51:50 +0300 Subject: [PATCH 536/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index da147ff98ca..7bbd3dbd762 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -366,7 +366,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); Преобразует входную строку в значение с типом данных [Decimal(P,S)](../../sql-reference/data-types/decimal.md). Семейство функций включает в себя: - `toDecimal32OrDefault(expr, S)` — возвращает значение типа `Decimal32(S)`. -- `toDecimal64OrDefault(expr, S)` — Возвращает значение типа `Decimal64(S)` data type. +- `toDecimal64OrDefault(expr, S)` — возвращает значение типа `Decimal64(S)`. - `toDecimal128OrDefault(expr, S)` — Возвращает значение типа `Decimal128(S)` data type. - `toDecimal256OrDefault(expr, S)` — Возвращает значение типа `Decimal256(S)` data type. From 1ecbe430659f8b8915dc1f130243849dc86ba449 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:52:08 +0300 Subject: [PATCH 537/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 7bbd3dbd762..9e7c0ba8e30 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -367,7 +367,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); - `toDecimal32OrDefault(expr, S)` — возвращает значение типа `Decimal32(S)`. - `toDecimal64OrDefault(expr, S)` — возвращает значение типа `Decimal64(S)`. -- `toDecimal128OrDefault(expr, S)` — Возвращает значение типа `Decimal128(S)` data type. +- `toDecimal128OrDefault(expr, S)` — возвращает значение типа `Decimal128(S)`. - `toDecimal256OrDefault(expr, S)` — Возвращает значение типа `Decimal256(S)` data type. Эти функции следует использовать вместо функций `toDecimal*()`, если при ошибке обработки входного значения вы хотите получать значение по умолчанию вместо исключения. From bf1e3b6b5bf08db2d75f5461e57b4eac4fe93a43 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Tue, 12 Oct 2021 23:52:16 +0300 Subject: [PATCH 538/950] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 9e7c0ba8e30..ca302d6c5de 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -368,7 +368,7 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); - `toDecimal32OrDefault(expr, S)` — возвращает значение типа `Decimal32(S)`. - `toDecimal64OrDefault(expr, S)` — возвращает значение типа `Decimal64(S)`. - `toDecimal128OrDefault(expr, S)` — возвращает значение типа `Decimal128(S)`. -- `toDecimal256OrDefault(expr, S)` — Возвращает значение типа `Decimal256(S)` data type. +- `toDecimal256OrDefault(expr, S)` — возвращает значение типа `Decimal256(S)`. Эти функции следует использовать вместо функций `toDecimal*()`, если при ошибке обработки входного значения вы хотите получать значение по умолчанию вместо исключения. From 4d2b34a7f0d346710172d71fc168cd0eb1edac17 Mon Sep 17 00:00:00 2001 From: Vitaly Orlov Date: Wed, 13 Oct 2021 00:08:00 +0300 Subject: [PATCH 539/950] Review fixes --- src/Functions/FunctionsJSON.h | 2 +- tests/queries/0_stateless/00918_json_functions.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 7407a3ec611..4f91440105d 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -1378,7 +1378,7 @@ public: auto & col_tuple = assert_cast(col_arr.getData()); auto & col_key = assert_cast(col_tuple.getColumn(0)); - for (auto [key, value] : object) + for (const auto & [key, value] : object) { col_key.insertData(key.data(), key.size()); } diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index bce84df08bf..87682587c8e 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -233,9 +233,9 @@ SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c": SELECT '--JSONExtractKeys--'; SELECT JSONExtractKeys('{"a": "hello", "b": [-100, 200.0, 300]}'); -SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); -SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); -SELECT JSONExtractKeysAndValuesRaw('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); +SELECT JSONExtractKeys('{"a": "hello", "b": [-100, 200.0, 300]}', 'b'); +SELECT JSONExtractKeys('{"a": "hello", "b": [-100, 200.0, 300]}', 'a'); +SELECT JSONExtractKeys('{"a": "hello", "b": [-100, 200.0, 300], "c":{"d":[121,144]}}', 'c'); SELECT '--const/non-const mixed--'; SELECT JSONExtractString('["a", "b", "c", "d", "e"]', idx) FROM (SELECT arrayJoin([1,2,3,4,5]) AS idx); From e9ce859b022751d7511da4730432db0f592eb96d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 23:45:23 +0300 Subject: [PATCH 540/950] Fix race between MOVE PARTITION and merges/mutations for MergeTree From time to time 00975_move_partition_merge_tree test got failed, the reason is that there is merge in progress that writes data to the source table again:
2021.10.12 13:20:40.243839 [ 26955 ] {2dbfea50-639e-49ac-9e82-933a00ea04a3} executeQuery: (from [::1]:38060) (comment: 00975_move_partition_merge_tree.sql) ALTER TABLE test_move_partition_src MOVE PARTITION 1 TO TABLE test_move_partition_dest; 2021.10.12 13:20:40.244482 [ 26955 ] {2dbfea50-639e-49ac-9e82-933a00ea04a3} test_t1tzb2.test_move_partition_dest (cab3b005-d54b-4cdc-8ab3-b005d54becdc): Cloning part /var/lib/clickhouse/store/467/467d7145-b47e-444e-867d-7145b47ea44e/1_2_2_0/ to /var/lib/clickhouse/store/cab/cab3b005-d54b-4cdc-8ab3-b005d54becdc/tmp_move_from_1_21_21_0 ... 2021.10.12 13:20:40.373487 [ 378 ] {} test_t1tzb2.test_move_partition_src (467d7145-b47e-444e-867d-7145b47ea44e) (MergerMutator): Merged 6 parts: from 1_2_2_0 to 1_11_11_0 ... 2021.10.12 13:20:40.379750 [ 26955 ] {2dbfea50-639e-49ac-9e82-933a00ea04a3} test_t1tzb2.test_move_partition_dest (cab3b005-d54b-4cdc-8ab3-b005d54becdc): Cloning part /var/lib/clickhouse/store/467/467d7145-b47e-444e-867d-7145b47ea44e/1_15_15_0/ to /var/lib/clickhouse/store/cab/cab3b005-d54b-4cdc-8ab3-b005d54becdc/tmp_move_from_1_28_28_0
And also remove cleaning of mutations since this will cause deadlock after doing MOVE PARTITION under currently_processing_in_background_mutex. CI: https://clickhouse-test-reports.s3.yandex.net/0/a59c6b1c8eb47ebf77189a491ee0c3980b38e91c/functional_stateless_tests_ Fixes: 00975_move_partition_merge_tree --- src/Storages/StorageMergeTree.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b710965271e..ab42da1dfa0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1447,6 +1447,11 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr local_context) { + /// MOVE PARTITION cannot be run in parallel with merges/mutations, + /// since otherwise there can be some merge/mutation in progress, + /// that will be created in the source table after MOVE PARTITION. + std::unique_lock background_lock(currently_processing_in_background_mutex); + auto lock1 = lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); auto lock2 = dest_table->lockForShare(local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); @@ -1509,7 +1514,6 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const transaction.commit(&lock); } - clearOldMutations(true); clearOldPartsFromFilesystem(); PartLog::addNewParts(getContext(), dst_parts, watch.elapsed()); From cd08a078d1ad1338481edb6cc30cbc1cb9ddbec7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 00:16:01 +0300 Subject: [PATCH 541/950] SQL user defined functions fix alias --- src/Interpreters/UserDefinedSQLFunctionVisitor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp index 6471f9cbf62..cc5db020387 100644 --- a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp @@ -93,6 +93,11 @@ ASTPtr UserDefinedSQLFunctionMatcher::tryToReplaceFunction(const ASTFunction & f } } + auto function_alias = function.tryGetAlias(); + + if (!function_alias.empty()) + function_body_to_update->setAlias(function_alias); + return function_body_to_update; } From e5bc573250d3d6938937739b05d6f8cf618722db Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 00:19:16 +0300 Subject: [PATCH 542/950] clickhouse-test: fix hung check in stress test by using system database --- tests/clickhouse-test | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 133c6f4e4e8..b2a9358371a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -77,6 +77,10 @@ def clickhouse_execute_http(base_args, query, timeout=30, settings=None, default params = { 'query': query, + # hung check in stress tests may remove the database, + # hence we should use 'system'. + 'database': 'system', + 'connect_timeout': timeout, 'receive_timeout': timeout, 'send_timeout': timeout, From dea5b5529feb27302104331d9812772d2ac1544d Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Tue, 12 Oct 2021 15:44:48 -0600 Subject: [PATCH 543/950] Updated adopters.md to include Sipfront --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index a6df18b323c..c511bd97a7c 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -115,6 +115,7 @@ toc_title: Adopters | seo.do | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | | SGK | Government Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | | Sina | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | +| Sipfront | Analytics | — | — | — | [Tweet, October 2021](https://twitter.com/andreasgranig/status/1446404332337913895?s=20) | | SMI2 | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) | | Spark New Zealand | Telecommunications | Security Operations | — | — | [Blog Post, Feb 2020](https://blog.n0p.me/2020/02/2020-02-05-dnsmonster/) | | Splitbee | Analytics | Main Product | — | — | [Blog Post, Mai 2021](https://splitbee.io/blog/new-pricing) | From c76d4c8a288fdf298636d1161741e77c901fe502 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Wed, 13 Oct 2021 00:49:26 +0300 Subject: [PATCH 544/950] Improved description. --- docs/ru/sql-reference/functions/type-conversion-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 772f07fe8ec..ae81e1e1ab6 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -279,9 +279,9 @@ SELECT toDate32OrNull('1955-01-01'), toDate32OrNull(''); └──────────────────────────────┴────────────────────┘ ``` -## toDate32OrDefault {#todate32-or-null} +## toDate32OrDefault {#todate32-or-default} -То же самое, что и [toDate32](#todate32), но возвращает значение по умолчанию, если получен недопустимый аргумент. +Конвертирует аргумент в значение типа [Date32](../../sql-reference/data-types/date32.md). Если значение выходит за границы диапазона, возвращается пограничное значение `Date32`. Если аргумент имеет тип [Date](../../sql-reference/data-types/date.md), учитываются границы типа `Date`. Возвращает значение по умолчанию, если получен недопустимый аргумент. **Пример** From c6ac339403aa198e1576950695e025e3da0313c7 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Wed, 13 Oct 2021 01:02:01 +0300 Subject: [PATCH 545/950] Updated desc-en --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 6385a1e650e..2abb78b1bb7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -281,7 +281,7 @@ Result: ## toDate32OrDefault {#todate32-or-default} -The same as [toDate32](#todate32) but returns default value if invalid argument is received. +Converts the argument to the [Date32](../../sql-reference/data-types/date32.md) data type. If the value is outside the range returns the border values supported by `Date32`. If the argument has [Date](../../sql-reference/data-types/date.md) type, borders of `Date` are taken into account. Returns default value if invalid argument is received. **Example** From 002c8d3a802a6db935e2ffd33ede7dbf9f6b0107 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 03:38:18 +0530 Subject: [PATCH 546/950] add disable vectorize flag due to llvm bug --- cmake/cpu_features.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index e77f330fbc0..f0750c69294 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -129,8 +129,8 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () -#we have to add -force-vector-width=16 due to llvm autovectorization bug with avx512 - set (TEST_FLAG "-mavx512f -mavx512bw -force-vector-width=16") +#Disable vectorize due to llvm autovectorization bug with avx512 + set (TEST_FLAG "-mavx512f -mavx512bw -fno-slp-vectorize -fno-vectorize") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include From 26dd0934d34782afcf22905d76b35740046f2546 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 13 Oct 2021 01:14:47 +0300 Subject: [PATCH 547/950] Update target.cmake --- cmake/target.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/target.cmake b/cmake/target.cmake index c1a34d0df13..3c02c4313f1 100644 --- a/cmake/target.cmake +++ b/cmake/target.cmake @@ -37,6 +37,7 @@ if (CMAKE_CROSSCOMPILING) set (ENABLE_PARQUET OFF CACHE INTERNAL "") set (ENABLE_ORC OFF CACHE INTERNAL "") set (ENABLE_GRPC OFF CACHE INTERNAL "") + set (ENABLE_EMBEDDED_COMPILER OFF CACHE INTERNAL "") else () message (FATAL_ERROR "Trying to cross-compile to unsupported system: ${CMAKE_SYSTEM_NAME}!") endif () From be5d49fd894351c150fe5e50c3d716cde0413a21 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Tue, 12 Oct 2021 16:16:23 -0600 Subject: [PATCH 548/950] Migrate changes in compiled css to sass source --- website/src/scss/components/_hero.scss | 28 ++++++++++++++++++++++++ website/src/scss/components/_navbar.scss | 8 ++++--- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/website/src/scss/components/_hero.scss b/website/src/scss/components/_hero.scss index 7c3bab209e4..61b22b3e32b 100644 --- a/website/src/scss/components/_hero.scss +++ b/website/src/scss/components/_hero.scss @@ -28,3 +28,31 @@ margin-bottom: -160px; padding-bottom: 160px; } + +.base-hero { + height:22.5vw; + max-height:324px; + min-height:280px; +} +.index-hero { + background-image:url('/images/backgrounds/bg-hero-home.svg'); + height:68vw; + max-height:980px; + max-width:2448px; + width:170vw; +} +.other-hero { + background-image: url('/images/backgrounds/bg-hero.svg'); + max-width: 2448px; + width: 170vw; +} +.bg-footer-cta { + background-image:url('/images/backgrounds/bg-footer-cta.svg'); + width:2448px; +} +.quickstart-bg { + background-image:url('/images/backgrounds/bg-quick-start.svg'); + height:40vw; + top:220px; + width:170vw; +} diff --git a/website/src/scss/components/_navbar.scss b/website/src/scss/components/_navbar.scss index 179e4ff1f60..53a834d2ed7 100644 --- a/website/src/scss/components/_navbar.scss +++ b/website/src/scss/components/_navbar.scss @@ -1,11 +1,13 @@ -.navbar { +.navbar-clickhouse { border-bottom: 4px solid $gray-100; height: 142px; > .container { flex-wrap: wrap; } +} +.navbar { &-super { flex-shrink: 0; width: 100%; @@ -38,8 +40,8 @@ } } - &-brand { - background: no-repeat url(#{"../images/logo.svg"}); + &-brand-clickhouse { + background: no-repeat url(#{"../images/logo-clickhouse.svg"}); background-size: contain; flex-shrink: 0; height: 28px; From f7091c4adb8ed89ff6fbd11627134fe10a6ab4cb Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Tue, 12 Oct 2021 16:16:39 -0600 Subject: [PATCH 549/950] Add greenhouse careers page --- docs/tools/webpack.config.js | 1 + docs/tools/website.py | 8 +- website/careers/index.html | 26 + website/css/bootstrap.css | 16035 +------------------- website/css/greenhouse.css | 1 + website/css/main.css | 1040 +- website/js/main.js | 159 +- website/src/js/main.js | 1 + website/src/js/utilities/greenhouse.js | 16 + website/src/scss/greenhouse.scss | 27 + website/templates/careers/greenhouse.html | 8 + website/templates/careers/hero.html | 10 + website/templates/careers/overview.html | 11 + website/templates/company/team.html | 2 +- 14 files changed, 112 insertions(+), 17233 deletions(-) create mode 100644 website/careers/index.html create mode 100644 website/css/greenhouse.css create mode 100644 website/src/js/utilities/greenhouse.js create mode 100644 website/src/scss/greenhouse.scss create mode 100644 website/templates/careers/greenhouse.html create mode 100644 website/templates/careers/hero.html create mode 100644 website/templates/careers/overview.html diff --git a/docs/tools/webpack.config.js b/docs/tools/webpack.config.js index e0dea964101..fcb3e7bf32d 100644 --- a/docs/tools/webpack.config.js +++ b/docs/tools/webpack.config.js @@ -14,6 +14,7 @@ module.exports = { entry: [ path.resolve(scssPath, 'bootstrap.scss'), + path.resolve(scssPath, 'greenhouse.scss'), path.resolve(scssPath, 'main.scss'), path.resolve(jsPath, 'main.js'), ], diff --git a/docs/tools/website.py b/docs/tools/website.py index 5e4f48e3441..4389ae9af3b 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -218,7 +218,7 @@ def minify_file(path, css_digest, js_digest): # TODO: restore cssmin # elif path.endswith('.css'): # content = cssmin.cssmin(content) -# TODO: restore jsmin +# TODO: restore jsmin # elif path.endswith('.js'): # content = jsmin.jsmin(content) with open(path, 'wb') as f: @@ -226,6 +226,12 @@ def minify_file(path, css_digest, js_digest): def minify_website(args): + # Output greenhouse css separately from main bundle to be included via the greenhouse iframe + command = f"cat '{args.website_dir}/css/greenhouse.css' > '{args.output_dir}/css/greenhouse.css'" + logging.info(command) + output = subprocess.check_output(command, shell=True) + logging.debug(output) + css_in = ' '.join(get_css_in(args)) css_out = f'{args.output_dir}/css/base.css' if args.minify: diff --git a/website/careers/index.html b/website/careers/index.html new file mode 100644 index 00000000000..14e23e3357c --- /dev/null +++ b/website/careers/index.html @@ -0,0 +1,26 @@ +{% set prefetch_items = [ + ('/docs/en/', 'document') +] %} + +{% extends "templates/base.html" %} + +{% block extra_meta %} +{% include "templates/common_fonts.html" %} +{% endblock %} + +{% block nav %} + +{% include "templates/global/nav.html" %} + +{% endblock %} + +{% block content %} + +{% include "templates/careers/hero.html" %} +{% include "templates/careers/overview.html" %} +{% include "templates/careers/greenhouse.html" %} + +{% include "templates/global/newsletter.html" %} +{% include "templates/global/github_stars.html" %} + +{% endblock %} diff --git a/website/css/bootstrap.css b/website/css/bootstrap.css index 92e98ef2c66..b65cbbfed01 100644 --- a/website/css/bootstrap.css +++ b/website/css/bootstrap.css @@ -1,16039 +1,6 @@ -@charset "UTF-8"; /*! * Bootstrap v4.4.1 (https://getbootstrap.com/) * Copyright 2011-2019 The Bootstrap Authors * Copyright 2011-2019 Twitter, Inc. * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) - */ -:root { - --blue: #007bff; - --indigo: #6610f2; - --purple: #6f42c1; - --pink: #e83e8c; - --red: #dc3545; - --orange: #fd7e14; - --yellow: #ffc107; - --green: #28a745; - --teal: #20c997; - --cyan: #17a2b8; - --white: #fff; - --gray: #6c757d; - --gray-dark: #343a40; - --brand-primary: #ffcc00; - --brand-secondary: #ff3939; - --primary-accent-yellow: #ffcc00; - --primary-accent-light-yellow: #fffaf0; - --primary-accent-blue: #257af4; - --primary-accent-light-blue: #e3f1fe; - --secondary-accent-orange: #ff8c00; - --secondary-accent-light-orange: #ffe4b5; - --secondary-accent-red: #ff3939; - --secondary-accent-light-red: #ffe4e1; - --primary: #ffcc00; - --secondary: #212529; - --success: #28a745; - --info: #17a2b8; - --warning: #ffc107; - --danger: #dc3545; - --light: #f1f6f9; - --dark: #495057; - --primary-light: #fffaf0; - --secondary-light: #fff; - --tertiary: #257af4; - --tertiary-light: #e3f1fe; - --white: #fff; - --black: #212529; - --blue: #257af4; - --light-blue: #e3f1fe; - --yellow: #ffcc00; - --light-yellow: #fffaf0; - --orange: #ff8c00; - --light-orange: #ffe4b5; - --red: #ff3939; - --light-red: #ffe4e1; - --medium: #d6dbdf; - --breakpoint-xxs: 0; - --breakpoint-xs: 400px; - --breakpoint-sm: 616px; - --breakpoint-md: 768px; - --breakpoint-lg: 980px; - --breakpoint-xl: 1240px; - --font-family-sans-serif: "Noto Sans", sans-serif; - --font-family-monospace: SFMono-Regular, Menlo, Monaco, Consolas, "Liberation Mono", "Courier New", monospace; -} - -*, -*::before, -*::after { - box-sizing: border-box; -} - -html { - font-family: sans-serif; - line-height: 1.15; - -webkit-text-size-adjust: 100%; - -webkit-tap-highlight-color: rgba(33, 37, 41, 0); -} - -article, aside, figcaption, figure, footer, header, hgroup, main, nav, section { - display: block; -} - -body { - margin: 0; - font-family: "Noto Sans", sans-serif; - font-size: 1rem; - font-weight: 400; - line-height: 1.5; - color: #212529; - text-align: left; - background-color: #fff; -} - -[tabindex="-1"]:focus:not(:focus-visible) { - outline: 0 !important; -} - -hr { - box-sizing: content-box; - height: 0; - overflow: visible; -} - -h1, h2, h3, h4, h5, h6 { - margin-top: 0; - margin-bottom: 16px; -} - -p { - margin-top: 0; - margin-bottom: 1rem; -} - -abbr[title], -abbr[data-original-title] { - text-decoration: underline; - -webkit-text-decoration: underline dotted; - text-decoration: underline dotted; - cursor: help; - border-bottom: 0; - -webkit-text-decoration-skip-ink: none; - text-decoration-skip-ink: none; -} - -address { - margin-bottom: 1rem; - font-style: normal; - line-height: inherit; -} - -ol, -ul, -dl { - margin-top: 0; - margin-bottom: 1rem; -} - -ol ol, -ul ul, -ol ul, -ul ol { - margin-bottom: 0; -} - -dt { - font-weight: 700; -} - -dd { - margin-bottom: 0.5rem; - margin-left: 0; -} - -blockquote { - margin: 0 0 1rem; -} - -b, -strong { - font-weight: bolder; -} - -small { - font-size: 80%; -} - -sub, -sup { - position: relative; - font-size: 75%; - line-height: 0; - vertical-align: baseline; -} - -sub { - bottom: -0.25em; -} - -sup { - top: -0.5em; -} - -a { - color: #ff8c00; - text-decoration: none; - background-color: transparent; -} -a:hover { - color: #ff8c00; - text-decoration: underline; -} - -a:not([href]) { - color: inherit; - text-decoration: none; -} -a:not([href]):hover { - color: inherit; - text-decoration: none; -} - -pre, -code, -kbd, -samp { - font-family: SFMono-Regular, Menlo, Monaco, Consolas, "Liberation Mono", "Courier New", monospace; - font-size: 1em; -} - -pre { - margin-top: 0; - margin-bottom: 1rem; - overflow: auto; -} - -figure { - margin: 0 0 1rem; -} - -img { - vertical-align: middle; - border-style: none; -} - -svg { - overflow: hidden; - vertical-align: middle; -} - -table { - border-collapse: collapse; -} - -caption { - padding-top: 0.75rem; - padding-bottom: 0.75rem; - color: #6c757d; - text-align: left; - caption-side: bottom; -} - -th { - text-align: inherit; -} - -label { - display: inline-block; - margin-bottom: 0.5rem; -} - -button { - border-radius: 0; -} - -button:focus { - outline: 1px dotted; - outline: 5px auto -webkit-focus-ring-color; -} - -input, -button, -select, -optgroup, -textarea { - margin: 0; - font-family: inherit; - font-size: inherit; - line-height: inherit; -} - -button, -input { - overflow: visible; -} - -button, -select { - text-transform: none; -} - -select { - word-wrap: normal; -} - -button, -[type=button], -[type=reset], -[type=submit] { - -webkit-appearance: button; -} - -button:not(:disabled), -[type=button]:not(:disabled), -[type=reset]:not(:disabled), -[type=submit]:not(:disabled) { - cursor: pointer; -} - -button::-moz-focus-inner, -[type=button]::-moz-focus-inner, -[type=reset]::-moz-focus-inner, -[type=submit]::-moz-focus-inner { - padding: 0; - border-style: none; -} - -input[type=radio], -input[type=checkbox] { - box-sizing: border-box; - padding: 0; -} - -input[type=date], -input[type=time], -input[type=datetime-local], -input[type=month] { - -webkit-appearance: listbox; -} - -textarea { - overflow: auto; - resize: vertical; -} - -fieldset { - min-width: 0; - padding: 0; - margin: 0; - border: 0; -} - -legend { - display: block; - width: 100%; - max-width: 100%; - padding: 0; - margin-bottom: 0.5rem; - font-size: 1.5rem; - line-height: inherit; - color: inherit; - white-space: normal; -} -@media (max-width: 1200px) { - legend { - font-size: calc(1.275rem + 0.3vw); - } -} - -progress { - vertical-align: baseline; -} - -[type=number]::-webkit-inner-spin-button, -[type=number]::-webkit-outer-spin-button { - height: auto; -} - -[type=search] { - outline-offset: -2px; - -webkit-appearance: none; -} - -[type=search]::-webkit-search-decoration { - -webkit-appearance: none; -} - -::-webkit-file-upload-button { - font: inherit; - -webkit-appearance: button; -} - -output { - display: inline-block; -} - -summary { - display: list-item; - cursor: pointer; -} - -template { - display: none; -} - -[hidden] { - display: none !important; -} - -h1, h2, h3, h4, h5, h6, -.h1, .h2, .h3, .h4, .h5, .h6 { - margin-bottom: 16px; - font-family: "Hind Siliguri", sans-serif; - font-weight: 500; - line-height: 1.125; -} - -h1, .h1 { - font-size: 2.5rem; -} -@media (max-width: 1200px) { - h1, .h1 { - font-size: calc(1.375rem + 1.5vw); - } -} - -h2, .h2 { - font-size: 2rem; -} -@media (max-width: 1200px) { - h2, .h2 { - font-size: calc(1.325rem + 0.9vw); - } -} - -h3, .h3 { - font-size: 1.75rem; -} -@media (max-width: 1200px) { - h3, .h3 { - font-size: calc(1.3rem + 0.6vw); - } -} - -h4, .h4 { - font-size: 1.5rem; -} -@media (max-width: 1200px) { - h4, .h4 { - font-size: calc(1.275rem + 0.3vw); - } -} - -h5, .h5 { - font-size: 1.125rem; -} - -h6, .h6 { - font-size: 0.875rem; -} - -.lead { - font-size: 1.375rem; - font-weight: 400; -} -@media (max-width: 1200px) { - .lead { - font-size: calc(1.2625rem + 0.15vw); - } -} - -.display-1 { - font-size: 4rem; - font-weight: 600; - line-height: 1.125; -} -@media (max-width: 1200px) { - .display-1 { - font-size: calc(1.525rem + 3.3vw); - } -} - -.display-2 { - font-size: 2.5rem; - font-weight: 600; - line-height: 1.125; -} -@media (max-width: 1200px) { - .display-2 { - font-size: calc(1.375rem + 1.5vw); - } -} - -.display-3 { - font-size: 2rem; - font-weight: 500; - line-height: 1.125; -} -@media (max-width: 1200px) { - .display-3 { - font-size: calc(1.325rem + 0.9vw); - } -} - -.display-4 { - font-size: 1.75rem; - font-weight: 500; - line-height: 1.125; -} -@media (max-width: 1200px) { - .display-4 { - font-size: calc(1.3rem + 0.6vw); - } -} - -hr { - margin-top: 8px; - margin-bottom: 8px; - border: 0; - border-top: 1px solid rgba(33, 37, 41, 0.1); -} - -small, -.small { - font-size: 80%; - font-weight: 400; -} - -mark, -.mark { - padding: 0.2em; - background-color: #fcf8e3; -} - -.list-unstyled { - padding-left: 0; - list-style: none; -} - -.list-inline { - padding-left: 0; - list-style: none; -} - -.list-inline-item { - display: inline-block; -} -.list-inline-item:not(:last-child) { - margin-right: 0.5rem; -} - -.initialism { - font-size: 90%; - text-transform: uppercase; -} - -.blockquote { - margin-bottom: 8px; - font-size: 1.25rem; -} - -.blockquote-footer { - display: block; - font-size: 80%; - color: #6c757d; -} -.blockquote-footer::before { - content: "— "; -} - -.img-fluid { - max-width: 100%; - height: auto; -} - -.img-thumbnail { - padding: 0.25rem; - background-color: #fff; - border: 1px solid #dee2e6; - border-radius: 8px; - max-width: 100%; - height: auto; -} - -.figure { - display: inline-block; -} - -.figure-img { - margin-bottom: 4px; - line-height: 1; -} - -.figure-caption { - font-size: 90%; - color: #6c757d; -} - -code { - font-size: 87.5%; - color: #e83e8c; - word-wrap: break-word; -} -a > code { - color: inherit; -} - -kbd { - padding: 0.2rem 0.4rem; - font-size: 87.5%; - color: #fff; - background-color: #495057; - border-radius: 8px; -} -kbd kbd { - padding: 0; - font-size: 100%; - font-weight: 700; -} - -pre { - display: block; - font-size: 87.5%; - color: #495057; -} -pre code { - font-size: inherit; - color: inherit; - word-break: normal; -} - -.pre-scrollable { - max-height: 340px; - overflow-y: scroll; -} - -.container { - width: 100%; - padding-right: 20px; - padding-left: 20px; - margin-right: auto; - margin-left: auto; -} -@media (min-width: 400px) { - .container { - max-width: 576px; - } -} -@media (min-width: 616px) { - .container { - max-width: 576px; - } -} -@media (min-width: 768px) { - .container { - max-width: 958px; - } -} -@media (min-width: 980px) { - .container { - max-width: 1008px; - } -} -@media (min-width: 1240px) { - .container { - max-width: 1118px; - } -} - -.container-fluid, .container-xl, .container-lg, .container-md, .container-sm, .container-xs { - width: 100%; - padding-right: 20px; - padding-left: 20px; - margin-right: auto; - margin-left: auto; -} - -@media (min-width: 400px) { - .container-xs, .container { - max-width: 576px; - } -} -@media (min-width: 616px) { - .container-sm, .container-xs, .container { - max-width: 576px; - } -} -@media (min-width: 768px) { - .container-md, .container-sm, .container-xs, .container { - max-width: 958px; - } -} -@media (min-width: 980px) { - .container-lg, .container-md, .container-sm, .container-xs, .container { - max-width: 1008px; - } -} -@media (min-width: 1240px) { - .container-xl, .container-lg, .container-md, .container-sm, .container-xs, .container { - max-width: 1118px; - } -} -.row { - display: flex; - flex-wrap: wrap; - margin-right: -20px; - margin-left: -20px; -} - -.no-gutters { - margin-right: 0; - margin-left: 0; -} -.no-gutters > .col, -.no-gutters > [class*=col-] { - padding-right: 0; - padding-left: 0; -} - -.col-xl, -.col-xl-auto, .col-xl-12, .col-xl-11, .col-xl-10, .col-xl-9, .col-xl-8, .col-xl-7, .col-xl-6, .col-xl-5, .col-xl-4, .col-xl-3, .col-xl-2, .col-xl-1, .col-lg, -.col-lg-auto, .col-lg-12, .col-lg-11, .col-lg-10, .col-lg-9, .col-lg-8, .col-lg-7, .col-lg-6, .col-lg-5, .col-lg-4, .col-lg-3, .col-lg-2, .col-lg-1, .col-md, -.col-md-auto, .col-md-12, .col-md-11, .col-md-10, .col-md-9, .col-md-8, .col-md-7, .col-md-6, .col-md-5, .col-md-4, .col-md-3, .col-md-2, .col-md-1, .col-sm, -.col-sm-auto, .col-sm-12, .col-sm-11, .col-sm-10, .col-sm-9, .col-sm-8, .col-sm-7, .col-sm-6, .col-sm-5, .col-sm-4, .col-sm-3, .col-sm-2, .col-sm-1, .col-xs, -.col-xs-auto, .col-xs-12, .col-xs-11, .col-xs-10, .col-xs-9, .col-xs-8, .col-xs-7, .col-xs-6, .col-xs-5, .col-xs-4, .col-xs-3, .col-xs-2, .col-xs-1, .col, -.col-auto, .col-12, .col-11, .col-10, .col-9, .col-8, .col-7, .col-6, .col-5, .col-4, .col-3, .col-2, .col-1 { - position: relative; - width: 100%; - padding-right: 20px; - padding-left: 20px; -} - -.col { - flex-basis: 0; - flex-grow: 1; - max-width: 100%; -} - -.row-cols-1 > * { - flex: 0 0 100%; - max-width: 100%; -} - -.row-cols-2 > * { - flex: 0 0 50%; - max-width: 50%; -} - -.row-cols-3 > * { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; -} - -.row-cols-4 > * { - flex: 0 0 25%; - max-width: 25%; -} - -.row-cols-5 > * { - flex: 0 0 20%; - max-width: 20%; -} - -.row-cols-6 > * { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; -} - -.col-auto { - flex: 0 0 auto; - width: auto; - max-width: 100%; -} - -.col-1 { - flex: 0 0 8.3333333333%; - max-width: 8.3333333333%; -} - -.col-2 { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; -} - -.col-3 { - flex: 0 0 25%; - max-width: 25%; -} - -.col-4 { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; -} - -.col-5 { - flex: 0 0 41.6666666667%; - max-width: 41.6666666667%; -} - -.col-6 { - flex: 0 0 50%; - max-width: 50%; -} - -.col-7 { - flex: 0 0 58.3333333333%; - max-width: 58.3333333333%; -} - -.col-8 { - flex: 0 0 66.6666666667%; - max-width: 66.6666666667%; -} - -.col-9 { - flex: 0 0 75%; - max-width: 75%; -} - -.col-10 { - flex: 0 0 83.3333333333%; - max-width: 83.3333333333%; -} - -.col-11 { - flex: 0 0 91.6666666667%; - max-width: 91.6666666667%; -} - -.col-12 { - flex: 0 0 100%; - max-width: 100%; -} - -.order-first { - order: -1; -} - -.order-last { - order: 13; -} - -.order-0 { - order: 0; -} - -.order-1 { - order: 1; -} - -.order-2 { - order: 2; -} - -.order-3 { - order: 3; -} - -.order-4 { - order: 4; -} - -.order-5 { - order: 5; -} - -.order-6 { - order: 6; -} - -.order-7 { - order: 7; -} - -.order-8 { - order: 8; -} - -.order-9 { - order: 9; -} - -.order-10 { - order: 10; -} - -.order-11 { - order: 11; -} - -.order-12 { - order: 12; -} - -.offset-1 { - margin-left: 8.3333333333%; -} - -.offset-2 { - margin-left: 16.6666666667%; -} - -.offset-3 { - margin-left: 25%; -} - -.offset-4 { - margin-left: 33.3333333333%; -} - -.offset-5 { - margin-left: 41.6666666667%; -} - -.offset-6 { - margin-left: 50%; -} - -.offset-7 { - margin-left: 58.3333333333%; -} - -.offset-8 { - margin-left: 66.6666666667%; -} - -.offset-9 { - margin-left: 75%; -} - -.offset-10 { - margin-left: 83.3333333333%; -} - -.offset-11 { - margin-left: 91.6666666667%; -} - -@media (min-width: 400px) { - .col-xs { - flex-basis: 0; - flex-grow: 1; - max-width: 100%; - } - - .row-cols-xs-1 > * { - flex: 0 0 100%; - max-width: 100%; - } - - .row-cols-xs-2 > * { - flex: 0 0 50%; - max-width: 50%; - } - - .row-cols-xs-3 > * { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .row-cols-xs-4 > * { - flex: 0 0 25%; - max-width: 25%; - } - - .row-cols-xs-5 > * { - flex: 0 0 20%; - max-width: 20%; - } - - .row-cols-xs-6 > * { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-xs-auto { - flex: 0 0 auto; - width: auto; - max-width: 100%; - } - - .col-xs-1 { - flex: 0 0 8.3333333333%; - max-width: 8.3333333333%; - } - - .col-xs-2 { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-xs-3 { - flex: 0 0 25%; - max-width: 25%; - } - - .col-xs-4 { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .col-xs-5 { - flex: 0 0 41.6666666667%; - max-width: 41.6666666667%; - } - - .col-xs-6 { - flex: 0 0 50%; - max-width: 50%; - } - - .col-xs-7 { - flex: 0 0 58.3333333333%; - max-width: 58.3333333333%; - } - - .col-xs-8 { - flex: 0 0 66.6666666667%; - max-width: 66.6666666667%; - } - - .col-xs-9 { - flex: 0 0 75%; - max-width: 75%; - } - - .col-xs-10 { - flex: 0 0 83.3333333333%; - max-width: 83.3333333333%; - } - - .col-xs-11 { - flex: 0 0 91.6666666667%; - max-width: 91.6666666667%; - } - - .col-xs-12 { - flex: 0 0 100%; - max-width: 100%; - } - - .order-xs-first { - order: -1; - } - - .order-xs-last { - order: 13; - } - - .order-xs-0 { - order: 0; - } - - .order-xs-1 { - order: 1; - } - - .order-xs-2 { - order: 2; - } - - .order-xs-3 { - order: 3; - } - - .order-xs-4 { - order: 4; - } - - .order-xs-5 { - order: 5; - } - - .order-xs-6 { - order: 6; - } - - .order-xs-7 { - order: 7; - } - - .order-xs-8 { - order: 8; - } - - .order-xs-9 { - order: 9; - } - - .order-xs-10 { - order: 10; - } - - .order-xs-11 { - order: 11; - } - - .order-xs-12 { - order: 12; - } - - .offset-xs-0 { - margin-left: 0; - } - - .offset-xs-1 { - margin-left: 8.3333333333%; - } - - .offset-xs-2 { - margin-left: 16.6666666667%; - } - - .offset-xs-3 { - margin-left: 25%; - } - - .offset-xs-4 { - margin-left: 33.3333333333%; - } - - .offset-xs-5 { - margin-left: 41.6666666667%; - } - - .offset-xs-6 { - margin-left: 50%; - } - - .offset-xs-7 { - margin-left: 58.3333333333%; - } - - .offset-xs-8 { - margin-left: 66.6666666667%; - } - - .offset-xs-9 { - margin-left: 75%; - } - - .offset-xs-10 { - margin-left: 83.3333333333%; - } - - .offset-xs-11 { - margin-left: 91.6666666667%; - } -} -@media (min-width: 616px) { - .col-sm { - flex-basis: 0; - flex-grow: 1; - max-width: 100%; - } - - .row-cols-sm-1 > * { - flex: 0 0 100%; - max-width: 100%; - } - - .row-cols-sm-2 > * { - flex: 0 0 50%; - max-width: 50%; - } - - .row-cols-sm-3 > * { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .row-cols-sm-4 > * { - flex: 0 0 25%; - max-width: 25%; - } - - .row-cols-sm-5 > * { - flex: 0 0 20%; - max-width: 20%; - } - - .row-cols-sm-6 > * { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-sm-auto { - flex: 0 0 auto; - width: auto; - max-width: 100%; - } - - .col-sm-1 { - flex: 0 0 8.3333333333%; - max-width: 8.3333333333%; - } - - .col-sm-2 { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-sm-3 { - flex: 0 0 25%; - max-width: 25%; - } - - .col-sm-4 { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .col-sm-5 { - flex: 0 0 41.6666666667%; - max-width: 41.6666666667%; - } - - .col-sm-6 { - flex: 0 0 50%; - max-width: 50%; - } - - .col-sm-7 { - flex: 0 0 58.3333333333%; - max-width: 58.3333333333%; - } - - .col-sm-8 { - flex: 0 0 66.6666666667%; - max-width: 66.6666666667%; - } - - .col-sm-9 { - flex: 0 0 75%; - max-width: 75%; - } - - .col-sm-10 { - flex: 0 0 83.3333333333%; - max-width: 83.3333333333%; - } - - .col-sm-11 { - flex: 0 0 91.6666666667%; - max-width: 91.6666666667%; - } - - .col-sm-12 { - flex: 0 0 100%; - max-width: 100%; - } - - .order-sm-first { - order: -1; - } - - .order-sm-last { - order: 13; - } - - .order-sm-0 { - order: 0; - } - - .order-sm-1 { - order: 1; - } - - .order-sm-2 { - order: 2; - } - - .order-sm-3 { - order: 3; - } - - .order-sm-4 { - order: 4; - } - - .order-sm-5 { - order: 5; - } - - .order-sm-6 { - order: 6; - } - - .order-sm-7 { - order: 7; - } - - .order-sm-8 { - order: 8; - } - - .order-sm-9 { - order: 9; - } - - .order-sm-10 { - order: 10; - } - - .order-sm-11 { - order: 11; - } - - .order-sm-12 { - order: 12; - } - - .offset-sm-0 { - margin-left: 0; - } - - .offset-sm-1 { - margin-left: 8.3333333333%; - } - - .offset-sm-2 { - margin-left: 16.6666666667%; - } - - .offset-sm-3 { - margin-left: 25%; - } - - .offset-sm-4 { - margin-left: 33.3333333333%; - } - - .offset-sm-5 { - margin-left: 41.6666666667%; - } - - .offset-sm-6 { - margin-left: 50%; - } - - .offset-sm-7 { - margin-left: 58.3333333333%; - } - - .offset-sm-8 { - margin-left: 66.6666666667%; - } - - .offset-sm-9 { - margin-left: 75%; - } - - .offset-sm-10 { - margin-left: 83.3333333333%; - } - - .offset-sm-11 { - margin-left: 91.6666666667%; - } -} -@media (min-width: 768px) { - .col-md { - flex-basis: 0; - flex-grow: 1; - max-width: 100%; - } - - .row-cols-md-1 > * { - flex: 0 0 100%; - max-width: 100%; - } - - .row-cols-md-2 > * { - flex: 0 0 50%; - max-width: 50%; - } - - .row-cols-md-3 > * { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .row-cols-md-4 > * { - flex: 0 0 25%; - max-width: 25%; - } - - .row-cols-md-5 > * { - flex: 0 0 20%; - max-width: 20%; - } - - .row-cols-md-6 > * { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-md-auto { - flex: 0 0 auto; - width: auto; - max-width: 100%; - } - - .col-md-1 { - flex: 0 0 8.3333333333%; - max-width: 8.3333333333%; - } - - .col-md-2 { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-md-3 { - flex: 0 0 25%; - max-width: 25%; - } - - .col-md-4 { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .col-md-5 { - flex: 0 0 41.6666666667%; - max-width: 41.6666666667%; - } - - .col-md-6 { - flex: 0 0 50%; - max-width: 50%; - } - - .col-md-7 { - flex: 0 0 58.3333333333%; - max-width: 58.3333333333%; - } - - .col-md-8 { - flex: 0 0 66.6666666667%; - max-width: 66.6666666667%; - } - - .col-md-9 { - flex: 0 0 75%; - max-width: 75%; - } - - .col-md-10 { - flex: 0 0 83.3333333333%; - max-width: 83.3333333333%; - } - - .col-md-11 { - flex: 0 0 91.6666666667%; - max-width: 91.6666666667%; - } - - .col-md-12 { - flex: 0 0 100%; - max-width: 100%; - } - - .order-md-first { - order: -1; - } - - .order-md-last { - order: 13; - } - - .order-md-0 { - order: 0; - } - - .order-md-1 { - order: 1; - } - - .order-md-2 { - order: 2; - } - - .order-md-3 { - order: 3; - } - - .order-md-4 { - order: 4; - } - - .order-md-5 { - order: 5; - } - - .order-md-6 { - order: 6; - } - - .order-md-7 { - order: 7; - } - - .order-md-8 { - order: 8; - } - - .order-md-9 { - order: 9; - } - - .order-md-10 { - order: 10; - } - - .order-md-11 { - order: 11; - } - - .order-md-12 { - order: 12; - } - - .offset-md-0 { - margin-left: 0; - } - - .offset-md-1 { - margin-left: 8.3333333333%; - } - - .offset-md-2 { - margin-left: 16.6666666667%; - } - - .offset-md-3 { - margin-left: 25%; - } - - .offset-md-4 { - margin-left: 33.3333333333%; - } - - .offset-md-5 { - margin-left: 41.6666666667%; - } - - .offset-md-6 { - margin-left: 50%; - } - - .offset-md-7 { - margin-left: 58.3333333333%; - } - - .offset-md-8 { - margin-left: 66.6666666667%; - } - - .offset-md-9 { - margin-left: 75%; - } - - .offset-md-10 { - margin-left: 83.3333333333%; - } - - .offset-md-11 { - margin-left: 91.6666666667%; - } -} -@media (min-width: 980px) { - .col-lg { - flex-basis: 0; - flex-grow: 1; - max-width: 100%; - } - - .row-cols-lg-1 > * { - flex: 0 0 100%; - max-width: 100%; - } - - .row-cols-lg-2 > * { - flex: 0 0 50%; - max-width: 50%; - } - - .row-cols-lg-3 > * { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .row-cols-lg-4 > * { - flex: 0 0 25%; - max-width: 25%; - } - - .row-cols-lg-5 > * { - flex: 0 0 20%; - max-width: 20%; - } - - .row-cols-lg-6 > * { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-lg-auto { - flex: 0 0 auto; - width: auto; - max-width: 100%; - } - - .col-lg-1 { - flex: 0 0 8.3333333333%; - max-width: 8.3333333333%; - } - - .col-lg-2 { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-lg-3 { - flex: 0 0 25%; - max-width: 25%; - } - - .col-lg-4 { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .col-lg-5 { - flex: 0 0 41.6666666667%; - max-width: 41.6666666667%; - } - - .col-lg-6 { - flex: 0 0 50%; - max-width: 50%; - } - - .col-lg-7 { - flex: 0 0 58.3333333333%; - max-width: 58.3333333333%; - } - - .col-lg-8 { - flex: 0 0 66.6666666667%; - max-width: 66.6666666667%; - } - - .col-lg-9 { - flex: 0 0 75%; - max-width: 75%; - } - - .col-lg-10 { - flex: 0 0 83.3333333333%; - max-width: 83.3333333333%; - } - - .col-lg-11 { - flex: 0 0 91.6666666667%; - max-width: 91.6666666667%; - } - - .col-lg-12 { - flex: 0 0 100%; - max-width: 100%; - } - - .order-lg-first { - order: -1; - } - - .order-lg-last { - order: 13; - } - - .order-lg-0 { - order: 0; - } - - .order-lg-1 { - order: 1; - } - - .order-lg-2 { - order: 2; - } - - .order-lg-3 { - order: 3; - } - - .order-lg-4 { - order: 4; - } - - .order-lg-5 { - order: 5; - } - - .order-lg-6 { - order: 6; - } - - .order-lg-7 { - order: 7; - } - - .order-lg-8 { - order: 8; - } - - .order-lg-9 { - order: 9; - } - - .order-lg-10 { - order: 10; - } - - .order-lg-11 { - order: 11; - } - - .order-lg-12 { - order: 12; - } - - .offset-lg-0 { - margin-left: 0; - } - - .offset-lg-1 { - margin-left: 8.3333333333%; - } - - .offset-lg-2 { - margin-left: 16.6666666667%; - } - - .offset-lg-3 { - margin-left: 25%; - } - - .offset-lg-4 { - margin-left: 33.3333333333%; - } - - .offset-lg-5 { - margin-left: 41.6666666667%; - } - - .offset-lg-6 { - margin-left: 50%; - } - - .offset-lg-7 { - margin-left: 58.3333333333%; - } - - .offset-lg-8 { - margin-left: 66.6666666667%; - } - - .offset-lg-9 { - margin-left: 75%; - } - - .offset-lg-10 { - margin-left: 83.3333333333%; - } - - .offset-lg-11 { - margin-left: 91.6666666667%; - } -} -@media (min-width: 1240px) { - .col-xl { - flex-basis: 0; - flex-grow: 1; - max-width: 100%; - } - - .row-cols-xl-1 > * { - flex: 0 0 100%; - max-width: 100%; - } - - .row-cols-xl-2 > * { - flex: 0 0 50%; - max-width: 50%; - } - - .row-cols-xl-3 > * { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .row-cols-xl-4 > * { - flex: 0 0 25%; - max-width: 25%; - } - - .row-cols-xl-5 > * { - flex: 0 0 20%; - max-width: 20%; - } - - .row-cols-xl-6 > * { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-xl-auto { - flex: 0 0 auto; - width: auto; - max-width: 100%; - } - - .col-xl-1 { - flex: 0 0 8.3333333333%; - max-width: 8.3333333333%; - } - - .col-xl-2 { - flex: 0 0 16.6666666667%; - max-width: 16.6666666667%; - } - - .col-xl-3 { - flex: 0 0 25%; - max-width: 25%; - } - - .col-xl-4 { - flex: 0 0 33.3333333333%; - max-width: 33.3333333333%; - } - - .col-xl-5 { - flex: 0 0 41.6666666667%; - max-width: 41.6666666667%; - } - - .col-xl-6 { - flex: 0 0 50%; - max-width: 50%; - } - - .col-xl-7 { - flex: 0 0 58.3333333333%; - max-width: 58.3333333333%; - } - - .col-xl-8 { - flex: 0 0 66.6666666667%; - max-width: 66.6666666667%; - } - - .col-xl-9 { - flex: 0 0 75%; - max-width: 75%; - } - - .col-xl-10 { - flex: 0 0 83.3333333333%; - max-width: 83.3333333333%; - } - - .col-xl-11 { - flex: 0 0 91.6666666667%; - max-width: 91.6666666667%; - } - - .col-xl-12 { - flex: 0 0 100%; - max-width: 100%; - } - - .order-xl-first { - order: -1; - } - - .order-xl-last { - order: 13; - } - - .order-xl-0 { - order: 0; - } - - .order-xl-1 { - order: 1; - } - - .order-xl-2 { - order: 2; - } - - .order-xl-3 { - order: 3; - } - - .order-xl-4 { - order: 4; - } - - .order-xl-5 { - order: 5; - } - - .order-xl-6 { - order: 6; - } - - .order-xl-7 { - order: 7; - } - - .order-xl-8 { - order: 8; - } - - .order-xl-9 { - order: 9; - } - - .order-xl-10 { - order: 10; - } - - .order-xl-11 { - order: 11; - } - - .order-xl-12 { - order: 12; - } - - .offset-xl-0 { - margin-left: 0; - } - - .offset-xl-1 { - margin-left: 8.3333333333%; - } - - .offset-xl-2 { - margin-left: 16.6666666667%; - } - - .offset-xl-3 { - margin-left: 25%; - } - - .offset-xl-4 { - margin-left: 33.3333333333%; - } - - .offset-xl-5 { - margin-left: 41.6666666667%; - } - - .offset-xl-6 { - margin-left: 50%; - } - - .offset-xl-7 { - margin-left: 58.3333333333%; - } - - .offset-xl-8 { - margin-left: 66.6666666667%; - } - - .offset-xl-9 { - margin-left: 75%; - } - - .offset-xl-10 { - margin-left: 83.3333333333%; - } - - .offset-xl-11 { - margin-left: 91.6666666667%; - } -} -.table { - width: 100%; - margin-bottom: 8px; - color: #212529; -} -.table th, -.table td { - padding: 0.75rem; - vertical-align: top; - border-top: 1px solid #d6dbdf; -} -.table thead th { - vertical-align: bottom; - border-bottom: 2px solid #d6dbdf; -} -.table tbody + tbody { - border-top: 2px solid #d6dbdf; -} - -.table-sm th, -.table-sm td { - padding: 0.3rem; -} - -.table-bordered { - border: 1px solid #d6dbdf; -} -.table-bordered th, -.table-bordered td { - border: 1px solid #d6dbdf; -} -.table-bordered thead th, -.table-bordered thead td { - border-bottom-width: 2px; -} - -.table-borderless th, -.table-borderless td, -.table-borderless thead th, -.table-borderless tbody + tbody { - border: 0; -} - -.table-striped tbody tr:nth-of-type(odd) { - background-color: rgba(33, 37, 41, 0.05); -} - -.table-hover tbody tr:hover { - color: #212529; - background-color: rgba(33, 37, 41, 0.075); -} - -.table-primary, -.table-primary > th, -.table-primary > td { - background-color: #fff1b8; -} -.table-primary th, -.table-primary td, -.table-primary thead th, -.table-primary tbody + tbody { - border-color: #ffe47a; -} - -.table-hover .table-primary:hover { - background-color: #ffec9f; -} -.table-hover .table-primary:hover > td, -.table-hover .table-primary:hover > th { - background-color: #ffec9f; -} - -.table-secondary, -.table-secondary > th, -.table-secondary > td { - background-color: #c1c2c3; -} -.table-secondary th, -.table-secondary td, -.table-secondary thead th, -.table-secondary tbody + tbody { - border-color: #8c8e90; -} - -.table-hover .table-secondary:hover { - background-color: #b4b5b6; -} -.table-hover .table-secondary:hover > td, -.table-hover .table-secondary:hover > th { - background-color: #b4b5b6; -} - -.table-success, -.table-success > th, -.table-success > td { - background-color: #c3e6cb; -} -.table-success th, -.table-success td, -.table-success thead th, -.table-success tbody + tbody { - border-color: #8fd19e; -} - -.table-hover .table-success:hover { - background-color: #b1dfbb; -} -.table-hover .table-success:hover > td, -.table-hover .table-success:hover > th { - background-color: #b1dfbb; -} - -.table-info, -.table-info > th, -.table-info > td { - background-color: #bee5eb; -} -.table-info th, -.table-info td, -.table-info thead th, -.table-info tbody + tbody { - border-color: #86cfda; -} - -.table-hover .table-info:hover { - background-color: #abdde5; -} -.table-hover .table-info:hover > td, -.table-hover .table-info:hover > th { - background-color: #abdde5; -} - -.table-warning, -.table-warning > th, -.table-warning > td { - background-color: #ffeeba; -} -.table-warning th, -.table-warning td, -.table-warning thead th, -.table-warning tbody + tbody { - border-color: #ffdf7e; -} - -.table-hover .table-warning:hover { - background-color: #ffe8a1; -} -.table-hover .table-warning:hover > td, -.table-hover .table-warning:hover > th { - background-color: #ffe8a1; -} - -.table-danger, -.table-danger > th, -.table-danger > td { - background-color: #f5c6cb; -} -.table-danger th, -.table-danger td, -.table-danger thead th, -.table-danger tbody + tbody { - border-color: #ed969e; -} - -.table-hover .table-danger:hover { - background-color: #f1b0b7; -} -.table-hover .table-danger:hover > td, -.table-hover .table-danger:hover > th { - background-color: #f1b0b7; -} - -.table-light, -.table-light > th, -.table-light > td { - background-color: #fbfcfd; -} -.table-light th, -.table-light td, -.table-light thead th, -.table-light tbody + tbody { - border-color: #f8fafc; -} - -.table-hover .table-light:hover { - background-color: #eaeff5; -} -.table-hover .table-light:hover > td, -.table-hover .table-light:hover > th { - background-color: #eaeff5; -} - -.table-dark, -.table-dark > th, -.table-dark > td { - background-color: #ccced0; -} -.table-dark th, -.table-dark td, -.table-dark thead th, -.table-dark tbody + tbody { - border-color: #a0a4a8; -} - -.table-hover .table-dark:hover { - background-color: #bfc1c4; -} -.table-hover .table-dark:hover > td, -.table-hover .table-dark:hover > th { - background-color: #bfc1c4; -} - -.table-primary-light, -.table-primary-light > th, -.table-primary-light > td { - background-color: #fffefb; -} -.table-primary-light th, -.table-primary-light td, -.table-primary-light thead th, -.table-primary-light tbody + tbody { - border-color: #fffcf7; -} - -.table-hover .table-primary-light:hover { - background-color: #fff8e2; -} -.table-hover .table-primary-light:hover > td, -.table-hover .table-primary-light:hover > th { - background-color: #fff8e2; -} - -.table-secondary-light, -.table-secondary-light > th, -.table-secondary-light > td { - background-color: white; -} -.table-secondary-light th, -.table-secondary-light td, -.table-secondary-light thead th, -.table-secondary-light tbody + tbody { - border-color: white; -} - -.table-hover .table-secondary-light:hover { - background-color: #f2f2f2; -} -.table-hover .table-secondary-light:hover > td, -.table-hover .table-secondary-light:hover > th { - background-color: #f2f2f2; -} - -.table-tertiary, -.table-tertiary > th, -.table-tertiary > td { - background-color: #c2dafc; -} -.table-tertiary th, -.table-tertiary td, -.table-tertiary thead th, -.table-tertiary tbody + tbody { - border-color: #8ebaf9; -} - -.table-hover .table-tertiary:hover { - background-color: #aacbfb; -} -.table-hover .table-tertiary:hover > td, -.table-hover .table-tertiary:hover > th { - background-color: #aacbfb; -} - -.table-tertiary-light, -.table-tertiary-light > th, -.table-tertiary-light > td { - background-color: #f7fbff; -} -.table-tertiary-light th, -.table-tertiary-light td, -.table-tertiary-light thead th, -.table-tertiary-light tbody + tbody { - border-color: #f0f8fe; -} - -.table-hover .table-tertiary-light:hover { - background-color: #deeeff; -} -.table-hover .table-tertiary-light:hover > td, -.table-hover .table-tertiary-light:hover > th { - background-color: #deeeff; -} - -.table-white, -.table-white > th, -.table-white > td { - background-color: white; -} -.table-white th, -.table-white td, -.table-white thead th, -.table-white tbody + tbody { - border-color: white; -} - -.table-hover .table-white:hover { - background-color: #f2f2f2; -} -.table-hover .table-white:hover > td, -.table-hover .table-white:hover > th { - background-color: #f2f2f2; -} - -.table-black, -.table-black > th, -.table-black > td { - background-color: #c1c2c3; -} -.table-black th, -.table-black td, -.table-black thead th, -.table-black tbody + tbody { - border-color: #8c8e90; -} - -.table-hover .table-black:hover { - background-color: #b4b5b6; -} -.table-hover .table-black:hover > td, -.table-hover .table-black:hover > th { - background-color: #b4b5b6; -} - -.table-blue, -.table-blue > th, -.table-blue > td { - background-color: #c2dafc; -} -.table-blue th, -.table-blue td, -.table-blue thead th, -.table-blue tbody + tbody { - border-color: #8ebaf9; -} - -.table-hover .table-blue:hover { - background-color: #aacbfb; -} -.table-hover .table-blue:hover > td, -.table-hover .table-blue:hover > th { - background-color: #aacbfb; -} - -.table-light-blue, -.table-light-blue > th, -.table-light-blue > td { - background-color: #f7fbff; -} -.table-light-blue th, -.table-light-blue td, -.table-light-blue thead th, -.table-light-blue tbody + tbody { - border-color: #f0f8fe; -} - -.table-hover .table-light-blue:hover { - background-color: #deeeff; -} -.table-hover .table-light-blue:hover > td, -.table-hover .table-light-blue:hover > th { - background-color: #deeeff; -} - -.table-yellow, -.table-yellow > th, -.table-yellow > td { - background-color: #fff1b8; -} -.table-yellow th, -.table-yellow td, -.table-yellow thead th, -.table-yellow tbody + tbody { - border-color: #ffe47a; -} - -.table-hover .table-yellow:hover { - background-color: #ffec9f; -} -.table-hover .table-yellow:hover > td, -.table-hover .table-yellow:hover > th { - background-color: #ffec9f; -} - -.table-light-yellow, -.table-light-yellow > th, -.table-light-yellow > td { - background-color: #fffefb; -} -.table-light-yellow th, -.table-light-yellow td, -.table-light-yellow thead th, -.table-light-yellow tbody + tbody { - border-color: #fffcf7; -} - -.table-hover .table-light-yellow:hover { - background-color: #fff8e2; -} -.table-hover .table-light-yellow:hover > td, -.table-hover .table-light-yellow:hover > th { - background-color: #fff8e2; -} - -.table-orange, -.table-orange > th, -.table-orange > td { - background-color: #ffdfb8; -} -.table-orange th, -.table-orange td, -.table-orange thead th, -.table-orange tbody + tbody { - border-color: #ffc37a; -} - -.table-hover .table-orange:hover { - background-color: #ffd49f; -} -.table-hover .table-orange:hover > td, -.table-hover .table-orange:hover > th { - background-color: #ffd49f; -} - -.table-light-orange, -.table-light-orange > th, -.table-light-orange > td { - background-color: #fff7ea; -} -.table-light-orange th, -.table-light-orange td, -.table-light-orange thead th, -.table-light-orange tbody + tbody { - border-color: #fff1d9; -} - -.table-hover .table-light-orange:hover { - background-color: #ffedd1; -} -.table-hover .table-light-orange:hover > td, -.table-hover .table-light-orange:hover > th { - background-color: #ffedd1; -} - -.table-red, -.table-red > th, -.table-red > td { - background-color: #ffc8c8; -} -.table-red th, -.table-red td, -.table-red thead th, -.table-red tbody + tbody { - border-color: #ff9898; -} - -.table-hover .table-red:hover { - background-color: #ffafaf; -} -.table-hover .table-red:hover > td, -.table-hover .table-red:hover > th { - background-color: #ffafaf; -} - -.table-light-red, -.table-light-red > th, -.table-light-red > td { - background-color: #fff7f7; -} -.table-light-red th, -.table-light-red td, -.table-light-red thead th, -.table-light-red tbody + tbody { - border-color: #fff1ef; -} - -.table-hover .table-light-red:hover { - background-color: #ffdede; -} -.table-hover .table-light-red:hover > td, -.table-hover .table-light-red:hover > th { - background-color: #ffdede; -} - -.table-medium, -.table-medium > th, -.table-medium > td { - background-color: #f4f5f6; -} -.table-medium th, -.table-medium td, -.table-medium thead th, -.table-medium tbody + tbody { - border-color: #eaecee; -} - -.table-hover .table-medium:hover { - background-color: #e6e8eb; -} -.table-hover .table-medium:hover > td, -.table-hover .table-medium:hover > th { - background-color: #e6e8eb; -} - -.table-active, -.table-active > th, -.table-active > td { - background-color: rgba(33, 37, 41, 0.075); -} - -.table-hover .table-active:hover { - background-color: rgba(22, 24, 27, 0.075); -} -.table-hover .table-active:hover > td, -.table-hover .table-active:hover > th { - background-color: rgba(22, 24, 27, 0.075); -} - -.table .thead-dark th { - color: #fff; - background-color: #343a40; - border-color: #454d55; -} -.table .thead-light th { - color: #6c757d; - background-color: #e9ecef; - border-color: #d6dbdf; -} - -.table-dark { - color: #fff; - background-color: #343a40; -} -.table-dark th, -.table-dark td, -.table-dark thead th { - border-color: #454d55; -} -.table-dark.table-bordered { - border: 0; -} -.table-dark.table-striped tbody tr:nth-of-type(odd) { - background-color: rgba(255, 255, 255, 0.05); -} -.table-dark.table-hover tbody tr:hover { - color: #fff; - background-color: rgba(255, 255, 255, 0.075); -} - -@media (max-width: 399.98px) { - .table-responsive-xs { - display: block; - width: 100%; - overflow-x: auto; - -webkit-overflow-scrolling: touch; - } - .table-responsive-xs > .table-bordered { - border: 0; - } -} -@media (max-width: 615.98px) { - .table-responsive-sm { - display: block; - width: 100%; - overflow-x: auto; - -webkit-overflow-scrolling: touch; - } - .table-responsive-sm > .table-bordered { - border: 0; - } -} -@media (max-width: 767.98px) { - .table-responsive-md { - display: block; - width: 100%; - overflow-x: auto; - -webkit-overflow-scrolling: touch; - } - .table-responsive-md > .table-bordered { - border: 0; - } -} -@media (max-width: 979.98px) { - .table-responsive-lg { - display: block; - width: 100%; - overflow-x: auto; - -webkit-overflow-scrolling: touch; - } - .table-responsive-lg > .table-bordered { - border: 0; - } -} -@media (max-width: 1239.98px) { - .table-responsive-xl { - display: block; - width: 100%; - overflow-x: auto; - -webkit-overflow-scrolling: touch; - } - .table-responsive-xl > .table-bordered { - border: 0; - } -} -.table-responsive { - display: block; - width: 100%; - overflow-x: auto; - -webkit-overflow-scrolling: touch; -} -.table-responsive > .table-bordered { - border: 0; -} - -.form-control { - display: block; - width: 100%; - height: calc(1.5em + 0.75rem + 2px); - padding: 0.375rem 0.75rem; - font-size: 1rem; - font-weight: 400; - line-height: 1.5; - color: #6c757d; - background-color: #fff; - background-clip: padding-box; - border: 1px solid #ced4da; - border-radius: 8px; - transition: border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; -} -@media (prefers-reduced-motion: reduce) { - .form-control { - transition: none; - } -} -.form-control::-ms-expand { - background-color: transparent; - border: 0; -} -.form-control:-moz-focusring { - color: transparent; - text-shadow: 0 0 0 #6c757d; -} -.form-control:focus { - color: #6c757d; - background-color: #fff; - border-color: #ffe680; - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.form-control::-moz-placeholder { - color: #6c757d; - opacity: 1; -} -.form-control:-ms-input-placeholder { - color: #6c757d; - opacity: 1; -} -.form-control::placeholder { - color: #6c757d; - opacity: 1; -} -.form-control:disabled, .form-control[readonly] { - background-color: #e9ecef; - opacity: 1; -} - -select.form-control:focus::-ms-value { - color: #6c757d; - background-color: #fff; -} - -.form-control-file, -.form-control-range { - display: block; - width: 100%; -} - -.col-form-label { - padding-top: calc(0.375rem + 1px); - padding-bottom: calc(0.375rem + 1px); - margin-bottom: 0; - font-size: inherit; - line-height: 1.5; -} - -.col-form-label-lg { - padding-top: calc(0.5rem + 1px); - padding-bottom: calc(0.5rem + 1px); - font-size: 1.125rem; - line-height: 1.5; -} - -.col-form-label-sm { - padding-top: calc(0.25rem + 1px); - padding-bottom: calc(0.25rem + 1px); - font-size: 0.875rem; - line-height: 1.5; -} - -.form-control-plaintext { - display: block; - width: 100%; - padding: 0.375rem 0; - margin-bottom: 0; - font-size: 1rem; - line-height: 1.5; - color: #212529; - background-color: transparent; - border: solid transparent; - border-width: 1px 0; -} -.form-control-plaintext.form-control-sm, .form-control-plaintext.form-control-lg { - padding-right: 0; - padding-left: 0; -} - -.form-control-sm { - height: calc(1.5em + 0.5rem + 2px); - padding: 0.25rem 0.5rem; - font-size: 0.875rem; - line-height: 1.5; - border-radius: 8px; -} - -.form-control-lg { - height: calc(1.5em + 1rem + 2px); - padding: 0.5rem 1rem; - font-size: 1.125rem; - line-height: 1.5; - border-radius: 8px; -} - -select.form-control[size], select.form-control[multiple] { - height: auto; -} - -textarea.form-control { - height: auto; -} - -.form-group { - margin-bottom: 1rem; -} - -.form-text { - display: block; - margin-top: 0.25rem; -} - -.form-row { - display: flex; - flex-wrap: wrap; - margin-right: -5px; - margin-left: -5px; -} -.form-row > .col, -.form-row > [class*=col-] { - padding-right: 5px; - padding-left: 5px; -} - -.form-check { - position: relative; - display: block; - padding-left: 1.25rem; -} - -.form-check-input { - position: absolute; - margin-top: 0.3rem; - margin-left: -1.25rem; -} -.form-check-input[disabled] ~ .form-check-label, .form-check-input:disabled ~ .form-check-label { - color: #6c757d; -} - -.form-check-label { - margin-bottom: 0; -} - -.form-check-inline { - display: inline-flex; - align-items: center; - padding-left: 0; - margin-right: 0.75rem; -} -.form-check-inline .form-check-input { - position: static; - margin-top: 0; - margin-right: 0.3125rem; - margin-left: 0; -} - -.valid-feedback { - display: none; - width: 100%; - margin-top: 0.25rem; - font-size: 80%; - color: #28a745; -} - -.valid-tooltip { - position: absolute; - top: 100%; - z-index: 5; - display: none; - max-width: 100%; - padding: 0.25rem 0.5rem; - margin-top: 0.1rem; - font-size: 0.875rem; - line-height: 1.5; - color: #fff; - background-color: rgba(40, 167, 69, 0.9); - border-radius: 8px; -} - -.was-validated :valid ~ .valid-feedback, -.was-validated :valid ~ .valid-tooltip, -.is-valid ~ .valid-feedback, -.is-valid ~ .valid-tooltip { - display: block; -} - -.was-validated .form-control:valid, .form-control.is-valid { - border-color: #28a745; - padding-right: calc(1.5em + 0.75rem); - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3e%3c/svg%3e"); - background-repeat: no-repeat; - background-position: right calc(0.375em + 0.1875rem) center; - background-size: calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); -} -.was-validated .form-control:valid:focus, .form-control.is-valid:focus { - border-color: #28a745; - box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); -} - -.was-validated textarea.form-control:valid, textarea.form-control.is-valid { - padding-right: calc(1.5em + 0.75rem); - background-position: top calc(0.375em + 0.1875rem) right calc(0.375em + 0.1875rem); -} - -.was-validated .custom-select:valid, .custom-select.is-valid { - border-color: #28a745; - padding-right: calc(0.75em + 2.3125rem); - background: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5' viewBox='0 0 4 5'%3e%3cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3e%3c/svg%3e") no-repeat right 0.75rem center/8px 10px, url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3e%3c/svg%3e") #fff no-repeat center right 1.75rem/calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); -} -.was-validated .custom-select:valid:focus, .custom-select.is-valid:focus { - border-color: #28a745; - box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); -} - -.was-validated .form-check-input:valid ~ .form-check-label, .form-check-input.is-valid ~ .form-check-label { - color: #28a745; -} -.was-validated .form-check-input:valid ~ .valid-feedback, -.was-validated .form-check-input:valid ~ .valid-tooltip, .form-check-input.is-valid ~ .valid-feedback, -.form-check-input.is-valid ~ .valid-tooltip { - display: block; -} - -.was-validated .custom-control-input:valid ~ .custom-control-label, .custom-control-input.is-valid ~ .custom-control-label { - color: #28a745; -} -.was-validated .custom-control-input:valid ~ .custom-control-label::before, .custom-control-input.is-valid ~ .custom-control-label::before { - border-color: #28a745; -} -.was-validated .custom-control-input:valid:checked ~ .custom-control-label::before, .custom-control-input.is-valid:checked ~ .custom-control-label::before { - border-color: #34ce57; - background-color: #34ce57; -} -.was-validated .custom-control-input:valid:focus ~ .custom-control-label::before, .custom-control-input.is-valid:focus ~ .custom-control-label::before { - box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); -} -.was-validated .custom-control-input:valid:focus:not(:checked) ~ .custom-control-label::before, .custom-control-input.is-valid:focus:not(:checked) ~ .custom-control-label::before { - border-color: #28a745; -} - -.was-validated .custom-file-input:valid ~ .custom-file-label, .custom-file-input.is-valid ~ .custom-file-label { - border-color: #28a745; -} -.was-validated .custom-file-input:valid:focus ~ .custom-file-label, .custom-file-input.is-valid:focus ~ .custom-file-label { - border-color: #28a745; - box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.25); -} - -.invalid-feedback { - display: none; - width: 100%; - margin-top: 0.25rem; - font-size: 80%; - color: #dc3545; -} - -.invalid-tooltip { - position: absolute; - top: 100%; - z-index: 5; - display: none; - max-width: 100%; - padding: 0.25rem 0.5rem; - margin-top: 0.1rem; - font-size: 0.875rem; - line-height: 1.5; - color: #fff; - background-color: rgba(220, 53, 69, 0.9); - border-radius: 8px; -} - -.was-validated :invalid ~ .invalid-feedback, -.was-validated :invalid ~ .invalid-tooltip, -.is-invalid ~ .invalid-feedback, -.is-invalid ~ .invalid-tooltip { - display: block; -} - -.was-validated .form-control:invalid, .form-control.is-invalid { - border-color: #dc3545; - padding-right: calc(1.5em + 0.75rem); - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545' viewBox='0 0 12 12'%3e%3ccircle cx='6' cy='6' r='4.5'/%3e%3cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3e%3ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3e%3c/svg%3e"); - background-repeat: no-repeat; - background-position: right calc(0.375em + 0.1875rem) center; - background-size: calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); -} -.was-validated .form-control:invalid:focus, .form-control.is-invalid:focus { - border-color: #dc3545; - box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); -} - -.was-validated textarea.form-control:invalid, textarea.form-control.is-invalid { - padding-right: calc(1.5em + 0.75rem); - background-position: top calc(0.375em + 0.1875rem) right calc(0.375em + 0.1875rem); -} - -.was-validated .custom-select:invalid, .custom-select.is-invalid { - border-color: #dc3545; - padding-right: calc(0.75em + 2.3125rem); - background: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5' viewBox='0 0 4 5'%3e%3cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3e%3c/svg%3e") no-repeat right 0.75rem center/8px 10px, url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545' viewBox='0 0 12 12'%3e%3ccircle cx='6' cy='6' r='4.5'/%3e%3cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3e%3ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3e%3c/svg%3e") #fff no-repeat center right 1.75rem/calc(0.75em + 0.375rem) calc(0.75em + 0.375rem); -} -.was-validated .custom-select:invalid:focus, .custom-select.is-invalid:focus { - border-color: #dc3545; - box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); -} - -.was-validated .form-check-input:invalid ~ .form-check-label, .form-check-input.is-invalid ~ .form-check-label { - color: #dc3545; -} -.was-validated .form-check-input:invalid ~ .invalid-feedback, -.was-validated .form-check-input:invalid ~ .invalid-tooltip, .form-check-input.is-invalid ~ .invalid-feedback, -.form-check-input.is-invalid ~ .invalid-tooltip { - display: block; -} - -.was-validated .custom-control-input:invalid ~ .custom-control-label, .custom-control-input.is-invalid ~ .custom-control-label { - color: #dc3545; -} -.was-validated .custom-control-input:invalid ~ .custom-control-label::before, .custom-control-input.is-invalid ~ .custom-control-label::before { - border-color: #dc3545; -} -.was-validated .custom-control-input:invalid:checked ~ .custom-control-label::before, .custom-control-input.is-invalid:checked ~ .custom-control-label::before { - border-color: #e4606d; - background-color: #e4606d; -} -.was-validated .custom-control-input:invalid:focus ~ .custom-control-label::before, .custom-control-input.is-invalid:focus ~ .custom-control-label::before { - box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); -} -.was-validated .custom-control-input:invalid:focus:not(:checked) ~ .custom-control-label::before, .custom-control-input.is-invalid:focus:not(:checked) ~ .custom-control-label::before { - border-color: #dc3545; -} - -.was-validated .custom-file-input:invalid ~ .custom-file-label, .custom-file-input.is-invalid ~ .custom-file-label { - border-color: #dc3545; -} -.was-validated .custom-file-input:invalid:focus ~ .custom-file-label, .custom-file-input.is-invalid:focus ~ .custom-file-label { - border-color: #dc3545; - box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.25); -} - -.form-inline { - display: flex; - flex-flow: row wrap; - align-items: center; -} -.form-inline .form-check { - width: 100%; -} -@media (min-width: 616px) { - .form-inline label { - display: flex; - align-items: center; - justify-content: center; - margin-bottom: 0; - } - .form-inline .form-group { - display: flex; - flex: 0 0 auto; - flex-flow: row wrap; - align-items: center; - margin-bottom: 0; - } - .form-inline .form-control { - display: inline-block; - width: auto; - vertical-align: middle; - } - .form-inline .form-control-plaintext { - display: inline-block; - } - .form-inline .input-group, -.form-inline .custom-select { - width: auto; - } - .form-inline .form-check { - display: flex; - align-items: center; - justify-content: center; - width: auto; - padding-left: 0; - } - .form-inline .form-check-input { - position: relative; - flex-shrink: 0; - margin-top: 0; - margin-right: 0.25rem; - margin-left: 0; - } - .form-inline .custom-control { - align-items: center; - justify-content: center; - } - .form-inline .custom-control-label { - margin-bottom: 0; - } -} - -.btn { - display: inline-block; - font-family: inherit; - font-weight: 700; - color: #212529; - text-align: center; - vertical-align: middle; - cursor: pointer; - -webkit-user-select: none; - -moz-user-select: none; - -ms-user-select: none; - user-select: none; - background-color: transparent; - border: 1px solid transparent; - padding: 12px 32px; - font-size: 0.875rem; - line-height: 20px; - border-radius: 8px; - transition: color 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; -} -@media (prefers-reduced-motion: reduce) { - .btn { - transition: none; - } -} -.btn:hover { - color: #212529; - text-decoration: none; -} -.btn:focus, .btn.focus { - outline: 0; - box-shadow: none; -} -.btn.disabled, .btn:disabled { - opacity: 0.65; -} -a.btn.disabled, -fieldset:disabled a.btn { - pointer-events: none; -} - -.btn-primary { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-primary:hover { - color: #495057; - background-color: #d9ad00; - border-color: #cca300; -} -.btn-primary:focus, .btn-primary.focus { - color: #495057; - background-color: #d9ad00; - border-color: #cca300; - box-shadow: 0 0 0 0 rgba(228, 185, 13, 0.5); -} -.btn-primary.disabled, .btn-primary:disabled { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-primary:not(:disabled):not(.disabled):active, .btn-primary:not(:disabled):not(.disabled).active, .show > .btn-primary.dropdown-toggle { - color: #495057; - background-color: #cca300; - border-color: #bf9900; -} -.btn-primary:not(:disabled):not(.disabled):active:focus, .btn-primary:not(:disabled):not(.disabled).active:focus, .show > .btn-primary.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 185, 13, 0.5); -} - -.btn-secondary { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-secondary:hover { - color: #fff; - background-color: #101214; - border-color: #0a0c0d; -} -.btn-secondary:focus, .btn-secondary.focus { - color: #fff; - background-color: #101214; - border-color: #0a0c0d; - box-shadow: 0 0 0 0 rgba(66, 70, 73, 0.5); -} -.btn-secondary.disabled, .btn-secondary:disabled { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-secondary:not(:disabled):not(.disabled):active, .btn-secondary:not(:disabled):not(.disabled).active, .show > .btn-secondary.dropdown-toggle { - color: #fff; - background-color: #0a0c0d; - border-color: #050506; -} -.btn-secondary:not(:disabled):not(.disabled):active:focus, .btn-secondary:not(:disabled):not(.disabled).active:focus, .show > .btn-secondary.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(66, 70, 73, 0.5); -} - -.btn-success { - color: #fff; - background-color: #28a745; - border-color: #28a745; -} -.btn-success:hover { - color: #fff; - background-color: #218838; - border-color: #1e7e34; -} -.btn-success:focus, .btn-success.focus { - color: #fff; - background-color: #218838; - border-color: #1e7e34; - box-shadow: 0 0 0 0 rgba(72, 180, 97, 0.5); -} -.btn-success.disabled, .btn-success:disabled { - color: #fff; - background-color: #28a745; - border-color: #28a745; -} -.btn-success:not(:disabled):not(.disabled):active, .btn-success:not(:disabled):not(.disabled).active, .show > .btn-success.dropdown-toggle { - color: #fff; - background-color: #1e7e34; - border-color: #1c7430; -} -.btn-success:not(:disabled):not(.disabled):active:focus, .btn-success:not(:disabled):not(.disabled).active:focus, .show > .btn-success.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(72, 180, 97, 0.5); -} - -.btn-info { - color: #fff; - background-color: #17a2b8; - border-color: #17a2b8; -} -.btn-info:hover { - color: #fff; - background-color: #138496; - border-color: #117a8b; -} -.btn-info:focus, .btn-info.focus { - color: #fff; - background-color: #138496; - border-color: #117a8b; - box-shadow: 0 0 0 0 rgba(58, 176, 195, 0.5); -} -.btn-info.disabled, .btn-info:disabled { - color: #fff; - background-color: #17a2b8; - border-color: #17a2b8; -} -.btn-info:not(:disabled):not(.disabled):active, .btn-info:not(:disabled):not(.disabled).active, .show > .btn-info.dropdown-toggle { - color: #fff; - background-color: #117a8b; - border-color: #10707f; -} -.btn-info:not(:disabled):not(.disabled):active:focus, .btn-info:not(:disabled):not(.disabled).active:focus, .show > .btn-info.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(58, 176, 195, 0.5); -} - -.btn-warning { - color: #495057; - background-color: #ffc107; - border-color: #ffc107; -} -.btn-warning:hover { - color: #495057; - background-color: #e0a800; - border-color: #d39e00; -} -.btn-warning:focus, .btn-warning.focus { - color: #495057; - background-color: #e0a800; - border-color: #d39e00; - box-shadow: 0 0 0 0 rgba(228, 176, 19, 0.5); -} -.btn-warning.disabled, .btn-warning:disabled { - color: #495057; - background-color: #ffc107; - border-color: #ffc107; -} -.btn-warning:not(:disabled):not(.disabled):active, .btn-warning:not(:disabled):not(.disabled).active, .show > .btn-warning.dropdown-toggle { - color: #495057; - background-color: #d39e00; - border-color: #c69500; -} -.btn-warning:not(:disabled):not(.disabled):active:focus, .btn-warning:not(:disabled):not(.disabled).active:focus, .show > .btn-warning.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 176, 19, 0.5); -} - -.btn-danger { - color: #fff; - background-color: #dc3545; - border-color: #dc3545; -} -.btn-danger:hover { - color: #fff; - background-color: #c82333; - border-color: #bd2130; -} -.btn-danger:focus, .btn-danger.focus { - color: #fff; - background-color: #c82333; - border-color: #bd2130; - box-shadow: 0 0 0 0 rgba(225, 83, 97, 0.5); -} -.btn-danger.disabled, .btn-danger:disabled { - color: #fff; - background-color: #dc3545; - border-color: #dc3545; -} -.btn-danger:not(:disabled):not(.disabled):active, .btn-danger:not(:disabled):not(.disabled).active, .show > .btn-danger.dropdown-toggle { - color: #fff; - background-color: #bd2130; - border-color: #b21f2d; -} -.btn-danger:not(:disabled):not(.disabled):active:focus, .btn-danger:not(:disabled):not(.disabled).active:focus, .show > .btn-danger.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(225, 83, 97, 0.5); -} - -.btn-light { - color: #495057; - background-color: #f1f6f9; - border-color: #f1f6f9; -} -.btn-light:hover { - color: #495057; - background-color: #d6e5ee; - border-color: #cddfea; -} -.btn-light:focus, .btn-light.focus { - color: #495057; - background-color: #d6e5ee; - border-color: #cddfea; - box-shadow: 0 0 0 0 rgba(216, 221, 225, 0.5); -} -.btn-light.disabled, .btn-light:disabled { - color: #495057; - background-color: #f1f6f9; - border-color: #f1f6f9; -} -.btn-light:not(:disabled):not(.disabled):active, .btn-light:not(:disabled):not(.disabled).active, .show > .btn-light.dropdown-toggle { - color: #495057; - background-color: #cddfea; - border-color: #c4d9e6; -} -.btn-light:not(:disabled):not(.disabled):active:focus, .btn-light:not(:disabled):not(.disabled).active:focus, .show > .btn-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(216, 221, 225, 0.5); -} - -.btn-dark { - color: #fff; - background-color: #495057; - border-color: #495057; -} -.btn-dark:hover { - color: #fff; - background-color: #383d42; - border-color: #32373b; -} -.btn-dark:focus, .btn-dark.focus { - color: #fff; - background-color: #383d42; - border-color: #32373b; - box-shadow: 0 0 0 0 rgba(100, 106, 112, 0.5); -} -.btn-dark.disabled, .btn-dark:disabled { - color: #fff; - background-color: #495057; - border-color: #495057; -} -.btn-dark:not(:disabled):not(.disabled):active, .btn-dark:not(:disabled):not(.disabled).active, .show > .btn-dark.dropdown-toggle { - color: #fff; - background-color: #32373b; - border-color: #2c3034; -} -.btn-dark:not(:disabled):not(.disabled):active:focus, .btn-dark:not(:disabled):not(.disabled).active:focus, .show > .btn-dark.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(100, 106, 112, 0.5); -} - -.btn-primary-light { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-primary-light:hover { - color: #495057; - background-color: #ffedca; - border-color: #ffe9bd; -} -.btn-primary-light:focus, .btn-primary-light.focus { - color: #495057; - background-color: #ffedca; - border-color: #ffe9bd; - box-shadow: 0 0 0 0 rgba(228, 225, 217, 0.5); -} -.btn-primary-light.disabled, .btn-primary-light:disabled { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-primary-light:not(:disabled):not(.disabled):active, .btn-primary-light:not(:disabled):not(.disabled).active, .show > .btn-primary-light.dropdown-toggle { - color: #495057; - background-color: #ffe9bd; - border-color: #ffe5b0; -} -.btn-primary-light:not(:disabled):not(.disabled):active:focus, .btn-primary-light:not(:disabled):not(.disabled).active:focus, .show > .btn-primary-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 225, 217, 0.5); -} - -.btn-secondary-light { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-secondary-light:hover { - color: #495057; - background-color: #ececec; - border-color: #e6e6e6; -} -.btn-secondary-light:focus, .btn-secondary-light.focus { - color: #495057; - background-color: #ececec; - border-color: #e6e6e6; - box-shadow: 0 0 0 0 rgba(228, 229, 230, 0.5); -} -.btn-secondary-light.disabled, .btn-secondary-light:disabled { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-secondary-light:not(:disabled):not(.disabled):active, .btn-secondary-light:not(:disabled):not(.disabled).active, .show > .btn-secondary-light.dropdown-toggle { - color: #495057; - background-color: #e6e6e6; - border-color: #dfdfdf; -} -.btn-secondary-light:not(:disabled):not(.disabled):active:focus, .btn-secondary-light:not(:disabled):not(.disabled).active:focus, .show > .btn-secondary-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 229, 230, 0.5); -} - -.btn-tertiary { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-tertiary:hover { - color: #fff; - background-color: #0c66e7; - border-color: #0b60db; -} -.btn-tertiary:focus, .btn-tertiary.focus { - color: #fff; - background-color: #0c66e7; - border-color: #0b60db; - box-shadow: 0 0 0 0 rgba(70, 142, 246, 0.5); -} -.btn-tertiary.disabled, .btn-tertiary:disabled { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-tertiary:not(:disabled):not(.disabled):active, .btn-tertiary:not(:disabled):not(.disabled).active, .show > .btn-tertiary.dropdown-toggle { - color: #fff; - background-color: #0b60db; - border-color: #0a5bcf; -} -.btn-tertiary:not(:disabled):not(.disabled):active:focus, .btn-tertiary:not(:disabled):not(.disabled).active:focus, .show > .btn-tertiary.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(70, 142, 246, 0.5); -} - -.btn-tertiary-light { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-tertiary-light:hover { - color: #495057; - background-color: #bedffd; - border-color: #b2d8fc; -} -.btn-tertiary-light:focus, .btn-tertiary-light.focus { - color: #495057; - background-color: #bedffd; - border-color: #b2d8fc; - box-shadow: 0 0 0 0 rgba(204, 217, 229, 0.5); -} -.btn-tertiary-light.disabled, .btn-tertiary-light:disabled { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-tertiary-light:not(:disabled):not(.disabled):active, .btn-tertiary-light:not(:disabled):not(.disabled).active, .show > .btn-tertiary-light.dropdown-toggle { - color: #495057; - background-color: #b2d8fc; - border-color: #a5d2fc; -} -.btn-tertiary-light:not(:disabled):not(.disabled):active:focus, .btn-tertiary-light:not(:disabled):not(.disabled).active:focus, .show > .btn-tertiary-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(204, 217, 229, 0.5); -} - -.btn-white { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-white:hover { - color: #495057; - background-color: #ececec; - border-color: #e6e6e6; -} -.btn-white:focus, .btn-white.focus { - color: #495057; - background-color: #ececec; - border-color: #e6e6e6; - box-shadow: 0 0 0 0 rgba(228, 229, 230, 0.5); -} -.btn-white.disabled, .btn-white:disabled { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-white:not(:disabled):not(.disabled):active, .btn-white:not(:disabled):not(.disabled).active, .show > .btn-white.dropdown-toggle { - color: #495057; - background-color: #e6e6e6; - border-color: #dfdfdf; -} -.btn-white:not(:disabled):not(.disabled):active:focus, .btn-white:not(:disabled):not(.disabled).active:focus, .show > .btn-white.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 229, 230, 0.5); -} - -.btn-black { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-black:hover { - color: #fff; - background-color: #101214; - border-color: #0a0c0d; -} -.btn-black:focus, .btn-black.focus { - color: #fff; - background-color: #101214; - border-color: #0a0c0d; - box-shadow: 0 0 0 0 rgba(66, 70, 73, 0.5); -} -.btn-black.disabled, .btn-black:disabled { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-black:not(:disabled):not(.disabled):active, .btn-black:not(:disabled):not(.disabled).active, .show > .btn-black.dropdown-toggle { - color: #fff; - background-color: #0a0c0d; - border-color: #050506; -} -.btn-black:not(:disabled):not(.disabled):active:focus, .btn-black:not(:disabled):not(.disabled).active:focus, .show > .btn-black.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(66, 70, 73, 0.5); -} - -.btn-blue { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-blue:hover { - color: #fff; - background-color: #0c66e7; - border-color: #0b60db; -} -.btn-blue:focus, .btn-blue.focus { - color: #fff; - background-color: #0c66e7; - border-color: #0b60db; - box-shadow: 0 0 0 0 rgba(70, 142, 246, 0.5); -} -.btn-blue.disabled, .btn-blue:disabled { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-blue:not(:disabled):not(.disabled):active, .btn-blue:not(:disabled):not(.disabled).active, .show > .btn-blue.dropdown-toggle { - color: #fff; - background-color: #0b60db; - border-color: #0a5bcf; -} -.btn-blue:not(:disabled):not(.disabled):active:focus, .btn-blue:not(:disabled):not(.disabled).active:focus, .show > .btn-blue.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(70, 142, 246, 0.5); -} - -.btn-light-blue { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-light-blue:hover { - color: #495057; - background-color: #bedffd; - border-color: #b2d8fc; -} -.btn-light-blue:focus, .btn-light-blue.focus { - color: #495057; - background-color: #bedffd; - border-color: #b2d8fc; - box-shadow: 0 0 0 0 rgba(204, 217, 229, 0.5); -} -.btn-light-blue.disabled, .btn-light-blue:disabled { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-light-blue:not(:disabled):not(.disabled):active, .btn-light-blue:not(:disabled):not(.disabled).active, .show > .btn-light-blue.dropdown-toggle { - color: #495057; - background-color: #b2d8fc; - border-color: #a5d2fc; -} -.btn-light-blue:not(:disabled):not(.disabled):active:focus, .btn-light-blue:not(:disabled):not(.disabled).active:focus, .show > .btn-light-blue.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(204, 217, 229, 0.5); -} - -.btn-yellow { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-yellow:hover { - color: #495057; - background-color: #d9ad00; - border-color: #cca300; -} -.btn-yellow:focus, .btn-yellow.focus { - color: #495057; - background-color: #d9ad00; - border-color: #cca300; - box-shadow: 0 0 0 0 rgba(228, 185, 13, 0.5); -} -.btn-yellow.disabled, .btn-yellow:disabled { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-yellow:not(:disabled):not(.disabled):active, .btn-yellow:not(:disabled):not(.disabled).active, .show > .btn-yellow.dropdown-toggle { - color: #495057; - background-color: #cca300; - border-color: #bf9900; -} -.btn-yellow:not(:disabled):not(.disabled):active:focus, .btn-yellow:not(:disabled):not(.disabled).active:focus, .show > .btn-yellow.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 185, 13, 0.5); -} - -.btn-light-yellow { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-light-yellow:hover { - color: #495057; - background-color: #ffedca; - border-color: #ffe9bd; -} -.btn-light-yellow:focus, .btn-light-yellow.focus { - color: #495057; - background-color: #ffedca; - border-color: #ffe9bd; - box-shadow: 0 0 0 0 rgba(228, 225, 217, 0.5); -} -.btn-light-yellow.disabled, .btn-light-yellow:disabled { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-light-yellow:not(:disabled):not(.disabled):active, .btn-light-yellow:not(:disabled):not(.disabled).active, .show > .btn-light-yellow.dropdown-toggle { - color: #495057; - background-color: #ffe9bd; - border-color: #ffe5b0; -} -.btn-light-yellow:not(:disabled):not(.disabled):active:focus, .btn-light-yellow:not(:disabled):not(.disabled).active:focus, .show > .btn-light-yellow.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 225, 217, 0.5); -} - -.btn-orange { - color: #495057; - background-color: #ff8c00; - border-color: #ff8c00; -} -.btn-orange:hover { - color: #fff; - background-color: #d97700; - border-color: #cc7000; -} -.btn-orange:focus, .btn-orange.focus { - color: #fff; - background-color: #d97700; - border-color: #cc7000; - box-shadow: 0 0 0 0 rgba(228, 131, 13, 0.5); -} -.btn-orange.disabled, .btn-orange:disabled { - color: #495057; - background-color: #ff8c00; - border-color: #ff8c00; -} -.btn-orange:not(:disabled):not(.disabled):active, .btn-orange:not(:disabled):not(.disabled).active, .show > .btn-orange.dropdown-toggle { - color: #fff; - background-color: #cc7000; - border-color: #bf6900; -} -.btn-orange:not(:disabled):not(.disabled):active:focus, .btn-orange:not(:disabled):not(.disabled).active:focus, .show > .btn-orange.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 131, 13, 0.5); -} - -.btn-light-orange { - color: #495057; - background-color: #ffe4b5; - border-color: #ffe4b5; -} -.btn-light-orange:hover { - color: #495057; - background-color: #ffd68f; - border-color: #ffd182; -} -.btn-light-orange:focus, .btn-light-orange.focus { - color: #495057; - background-color: #ffd68f; - border-color: #ffd182; - box-shadow: 0 0 0 0 rgba(228, 206, 167, 0.5); -} -.btn-light-orange.disabled, .btn-light-orange:disabled { - color: #495057; - background-color: #ffe4b5; - border-color: #ffe4b5; -} -.btn-light-orange:not(:disabled):not(.disabled):active, .btn-light-orange:not(:disabled):not(.disabled).active, .show > .btn-light-orange.dropdown-toggle { - color: #495057; - background-color: #ffd182; - border-color: #ffcd75; -} -.btn-light-orange:not(:disabled):not(.disabled):active:focus, .btn-light-orange:not(:disabled):not(.disabled).active:focus, .show > .btn-light-orange.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 206, 167, 0.5); -} - -.btn-red { - color: #fff; - background-color: #ff3939; - border-color: #ff3939; -} -.btn-red:hover { - color: #fff; - background-color: #ff1313; - border-color: #ff0606; -} -.btn-red:focus, .btn-red.focus { - color: #fff; - background-color: #ff1313; - border-color: #ff0606; - box-shadow: 0 0 0 0 rgba(255, 87, 87, 0.5); -} -.btn-red.disabled, .btn-red:disabled { - color: #fff; - background-color: #ff3939; - border-color: #ff3939; -} -.btn-red:not(:disabled):not(.disabled):active, .btn-red:not(:disabled):not(.disabled).active, .show > .btn-red.dropdown-toggle { - color: #fff; - background-color: #ff0606; - border-color: #f80000; -} -.btn-red:not(:disabled):not(.disabled):active:focus, .btn-red:not(:disabled):not(.disabled).active:focus, .show > .btn-red.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 87, 87, 0.5); -} - -.btn-light-red { - color: #495057; - background-color: #ffe4e1; - border-color: #ffe4e1; -} -.btn-light-red:hover { - color: #495057; - background-color: #ffc2bb; - border-color: #ffb6ae; -} -.btn-light-red:focus, .btn-light-red.focus { - color: #495057; - background-color: #ffc2bb; - border-color: #ffb6ae; - box-shadow: 0 0 0 0 rgba(228, 206, 204, 0.5); -} -.btn-light-red.disabled, .btn-light-red:disabled { - color: #495057; - background-color: #ffe4e1; - border-color: #ffe4e1; -} -.btn-light-red:not(:disabled):not(.disabled):active, .btn-light-red:not(:disabled):not(.disabled).active, .show > .btn-light-red.dropdown-toggle { - color: #495057; - background-color: #ffb6ae; - border-color: #ffaba1; -} -.btn-light-red:not(:disabled):not(.disabled):active:focus, .btn-light-red:not(:disabled):not(.disabled).active:focus, .show > .btn-light-red.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(228, 206, 204, 0.5); -} - -.btn-medium { - color: #495057; - background-color: #d6dbdf; - border-color: #d6dbdf; -} -.btn-medium:hover { - color: #495057; - background-color: #c1c8ce; - border-color: #b9c2c9; -} -.btn-medium:focus, .btn-medium.focus { - color: #495057; - background-color: #c1c8ce; - border-color: #b9c2c9; - box-shadow: 0 0 0 0 rgba(193, 198, 203, 0.5); -} -.btn-medium.disabled, .btn-medium:disabled { - color: #495057; - background-color: #d6dbdf; - border-color: #d6dbdf; -} -.btn-medium:not(:disabled):not(.disabled):active, .btn-medium:not(:disabled):not(.disabled).active, .show > .btn-medium.dropdown-toggle { - color: #495057; - background-color: #b9c2c9; - border-color: #b2bcc3; -} -.btn-medium:not(:disabled):not(.disabled):active:focus, .btn-medium:not(:disabled):not(.disabled).active:focus, .show > .btn-medium.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(193, 198, 203, 0.5); -} - -.btn-outline-primary { - color: #ffcc00; - border-color: #ffcc00; -} -.btn-outline-primary:hover { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-outline-primary:focus, .btn-outline-primary.focus { - box-shadow: 0 0 0 0 rgba(255, 204, 0, 0.5); -} -.btn-outline-primary.disabled, .btn-outline-primary:disabled { - color: #ffcc00; - background-color: transparent; -} -.btn-outline-primary:not(:disabled):not(.disabled):active, .btn-outline-primary:not(:disabled):not(.disabled).active, .show > .btn-outline-primary.dropdown-toggle { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-outline-primary:not(:disabled):not(.disabled):active:focus, .btn-outline-primary:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-primary.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 204, 0, 0.5); -} - -.btn-outline-secondary { - color: #212529; - border-color: #212529; -} -.btn-outline-secondary:hover { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-outline-secondary:focus, .btn-outline-secondary.focus { - box-shadow: 0 0 0 0 rgba(33, 37, 41, 0.5); -} -.btn-outline-secondary.disabled, .btn-outline-secondary:disabled { - color: #212529; - background-color: transparent; -} -.btn-outline-secondary:not(:disabled):not(.disabled):active, .btn-outline-secondary:not(:disabled):not(.disabled).active, .show > .btn-outline-secondary.dropdown-toggle { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-outline-secondary:not(:disabled):not(.disabled):active:focus, .btn-outline-secondary:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-secondary.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(33, 37, 41, 0.5); -} - -.btn-outline-success { - color: #28a745; - border-color: #28a745; -} -.btn-outline-success:hover { - color: #fff; - background-color: #28a745; - border-color: #28a745; -} -.btn-outline-success:focus, .btn-outline-success.focus { - box-shadow: 0 0 0 0 rgba(40, 167, 69, 0.5); -} -.btn-outline-success.disabled, .btn-outline-success:disabled { - color: #28a745; - background-color: transparent; -} -.btn-outline-success:not(:disabled):not(.disabled):active, .btn-outline-success:not(:disabled):not(.disabled).active, .show > .btn-outline-success.dropdown-toggle { - color: #fff; - background-color: #28a745; - border-color: #28a745; -} -.btn-outline-success:not(:disabled):not(.disabled):active:focus, .btn-outline-success:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-success.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(40, 167, 69, 0.5); -} - -.btn-outline-info { - color: #17a2b8; - border-color: #17a2b8; -} -.btn-outline-info:hover { - color: #fff; - background-color: #17a2b8; - border-color: #17a2b8; -} -.btn-outline-info:focus, .btn-outline-info.focus { - box-shadow: 0 0 0 0 rgba(23, 162, 184, 0.5); -} -.btn-outline-info.disabled, .btn-outline-info:disabled { - color: #17a2b8; - background-color: transparent; -} -.btn-outline-info:not(:disabled):not(.disabled):active, .btn-outline-info:not(:disabled):not(.disabled).active, .show > .btn-outline-info.dropdown-toggle { - color: #fff; - background-color: #17a2b8; - border-color: #17a2b8; -} -.btn-outline-info:not(:disabled):not(.disabled):active:focus, .btn-outline-info:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-info.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(23, 162, 184, 0.5); -} - -.btn-outline-warning { - color: #ffc107; - border-color: #ffc107; -} -.btn-outline-warning:hover { - color: #495057; - background-color: #ffc107; - border-color: #ffc107; -} -.btn-outline-warning:focus, .btn-outline-warning.focus { - box-shadow: 0 0 0 0 rgba(255, 193, 7, 0.5); -} -.btn-outline-warning.disabled, .btn-outline-warning:disabled { - color: #ffc107; - background-color: transparent; -} -.btn-outline-warning:not(:disabled):not(.disabled):active, .btn-outline-warning:not(:disabled):not(.disabled).active, .show > .btn-outline-warning.dropdown-toggle { - color: #495057; - background-color: #ffc107; - border-color: #ffc107; -} -.btn-outline-warning:not(:disabled):not(.disabled):active:focus, .btn-outline-warning:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-warning.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 193, 7, 0.5); -} - -.btn-outline-danger { - color: #dc3545; - border-color: #dc3545; -} -.btn-outline-danger:hover { - color: #fff; - background-color: #dc3545; - border-color: #dc3545; -} -.btn-outline-danger:focus, .btn-outline-danger.focus { - box-shadow: 0 0 0 0 rgba(220, 53, 69, 0.5); -} -.btn-outline-danger.disabled, .btn-outline-danger:disabled { - color: #dc3545; - background-color: transparent; -} -.btn-outline-danger:not(:disabled):not(.disabled):active, .btn-outline-danger:not(:disabled):not(.disabled).active, .show > .btn-outline-danger.dropdown-toggle { - color: #fff; - background-color: #dc3545; - border-color: #dc3545; -} -.btn-outline-danger:not(:disabled):not(.disabled):active:focus, .btn-outline-danger:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-danger.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(220, 53, 69, 0.5); -} - -.btn-outline-light { - color: #f1f6f9; - border-color: #f1f6f9; -} -.btn-outline-light:hover { - color: #495057; - background-color: #f1f6f9; - border-color: #f1f6f9; -} -.btn-outline-light:focus, .btn-outline-light.focus { - box-shadow: 0 0 0 0 rgba(241, 246, 249, 0.5); -} -.btn-outline-light.disabled, .btn-outline-light:disabled { - color: #f1f6f9; - background-color: transparent; -} -.btn-outline-light:not(:disabled):not(.disabled):active, .btn-outline-light:not(:disabled):not(.disabled).active, .show > .btn-outline-light.dropdown-toggle { - color: #495057; - background-color: #f1f6f9; - border-color: #f1f6f9; -} -.btn-outline-light:not(:disabled):not(.disabled):active:focus, .btn-outline-light:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(241, 246, 249, 0.5); -} - -.btn-outline-dark { - color: #495057; - border-color: #495057; -} -.btn-outline-dark:hover { - color: #fff; - background-color: #495057; - border-color: #495057; -} -.btn-outline-dark:focus, .btn-outline-dark.focus { - box-shadow: 0 0 0 0 rgba(73, 80, 87, 0.5); -} -.btn-outline-dark.disabled, .btn-outline-dark:disabled { - color: #495057; - background-color: transparent; -} -.btn-outline-dark:not(:disabled):not(.disabled):active, .btn-outline-dark:not(:disabled):not(.disabled).active, .show > .btn-outline-dark.dropdown-toggle { - color: #fff; - background-color: #495057; - border-color: #495057; -} -.btn-outline-dark:not(:disabled):not(.disabled):active:focus, .btn-outline-dark:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-dark.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(73, 80, 87, 0.5); -} - -.btn-outline-primary-light { - color: #fffaf0; - border-color: #fffaf0; -} -.btn-outline-primary-light:hover { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-outline-primary-light:focus, .btn-outline-primary-light.focus { - box-shadow: 0 0 0 0 rgba(255, 250, 240, 0.5); -} -.btn-outline-primary-light.disabled, .btn-outline-primary-light:disabled { - color: #fffaf0; - background-color: transparent; -} -.btn-outline-primary-light:not(:disabled):not(.disabled):active, .btn-outline-primary-light:not(:disabled):not(.disabled).active, .show > .btn-outline-primary-light.dropdown-toggle { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-outline-primary-light:not(:disabled):not(.disabled):active:focus, .btn-outline-primary-light:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-primary-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 250, 240, 0.5); -} - -.btn-outline-secondary-light { - color: #fff; - border-color: #fff; -} -.btn-outline-secondary-light:hover { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-outline-secondary-light:focus, .btn-outline-secondary-light.focus { - box-shadow: 0 0 0 0 rgba(255, 255, 255, 0.5); -} -.btn-outline-secondary-light.disabled, .btn-outline-secondary-light:disabled { - color: #fff; - background-color: transparent; -} -.btn-outline-secondary-light:not(:disabled):not(.disabled):active, .btn-outline-secondary-light:not(:disabled):not(.disabled).active, .show > .btn-outline-secondary-light.dropdown-toggle { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-outline-secondary-light:not(:disabled):not(.disabled):active:focus, .btn-outline-secondary-light:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-secondary-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 255, 255, 0.5); -} - -.btn-outline-tertiary { - color: #257af4; - border-color: #257af4; -} -.btn-outline-tertiary:hover { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-outline-tertiary:focus, .btn-outline-tertiary.focus { - box-shadow: 0 0 0 0 rgba(37, 122, 244, 0.5); -} -.btn-outline-tertiary.disabled, .btn-outline-tertiary:disabled { - color: #257af4; - background-color: transparent; -} -.btn-outline-tertiary:not(:disabled):not(.disabled):active, .btn-outline-tertiary:not(:disabled):not(.disabled).active, .show > .btn-outline-tertiary.dropdown-toggle { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-outline-tertiary:not(:disabled):not(.disabled):active:focus, .btn-outline-tertiary:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-tertiary.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(37, 122, 244, 0.5); -} - -.btn-outline-tertiary-light { - color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-outline-tertiary-light:hover { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-outline-tertiary-light:focus, .btn-outline-tertiary-light.focus { - box-shadow: 0 0 0 0 rgba(227, 241, 254, 0.5); -} -.btn-outline-tertiary-light.disabled, .btn-outline-tertiary-light:disabled { - color: #e3f1fe; - background-color: transparent; -} -.btn-outline-tertiary-light:not(:disabled):not(.disabled):active, .btn-outline-tertiary-light:not(:disabled):not(.disabled).active, .show > .btn-outline-tertiary-light.dropdown-toggle { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-outline-tertiary-light:not(:disabled):not(.disabled):active:focus, .btn-outline-tertiary-light:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-tertiary-light.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(227, 241, 254, 0.5); -} - -.btn-outline-white { - color: #fff; - border-color: #fff; -} -.btn-outline-white:hover { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-outline-white:focus, .btn-outline-white.focus { - box-shadow: 0 0 0 0 rgba(255, 255, 255, 0.5); -} -.btn-outline-white.disabled, .btn-outline-white:disabled { - color: #fff; - background-color: transparent; -} -.btn-outline-white:not(:disabled):not(.disabled):active, .btn-outline-white:not(:disabled):not(.disabled).active, .show > .btn-outline-white.dropdown-toggle { - color: #495057; - background-color: #fff; - border-color: #fff; -} -.btn-outline-white:not(:disabled):not(.disabled):active:focus, .btn-outline-white:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-white.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 255, 255, 0.5); -} - -.btn-outline-black { - color: #212529; - border-color: #212529; -} -.btn-outline-black:hover { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-outline-black:focus, .btn-outline-black.focus { - box-shadow: 0 0 0 0 rgba(33, 37, 41, 0.5); -} -.btn-outline-black.disabled, .btn-outline-black:disabled { - color: #212529; - background-color: transparent; -} -.btn-outline-black:not(:disabled):not(.disabled):active, .btn-outline-black:not(:disabled):not(.disabled).active, .show > .btn-outline-black.dropdown-toggle { - color: #fff; - background-color: #212529; - border-color: #212529; -} -.btn-outline-black:not(:disabled):not(.disabled):active:focus, .btn-outline-black:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-black.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(33, 37, 41, 0.5); -} - -.btn-outline-blue { - color: #257af4; - border-color: #257af4; -} -.btn-outline-blue:hover { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-outline-blue:focus, .btn-outline-blue.focus { - box-shadow: 0 0 0 0 rgba(37, 122, 244, 0.5); -} -.btn-outline-blue.disabled, .btn-outline-blue:disabled { - color: #257af4; - background-color: transparent; -} -.btn-outline-blue:not(:disabled):not(.disabled):active, .btn-outline-blue:not(:disabled):not(.disabled).active, .show > .btn-outline-blue.dropdown-toggle { - color: #fff; - background-color: #257af4; - border-color: #257af4; -} -.btn-outline-blue:not(:disabled):not(.disabled):active:focus, .btn-outline-blue:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-blue.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(37, 122, 244, 0.5); -} - -.btn-outline-light-blue { - color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-outline-light-blue:hover { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-outline-light-blue:focus, .btn-outline-light-blue.focus { - box-shadow: 0 0 0 0 rgba(227, 241, 254, 0.5); -} -.btn-outline-light-blue.disabled, .btn-outline-light-blue:disabled { - color: #e3f1fe; - background-color: transparent; -} -.btn-outline-light-blue:not(:disabled):not(.disabled):active, .btn-outline-light-blue:not(:disabled):not(.disabled).active, .show > .btn-outline-light-blue.dropdown-toggle { - color: #495057; - background-color: #e3f1fe; - border-color: #e3f1fe; -} -.btn-outline-light-blue:not(:disabled):not(.disabled):active:focus, .btn-outline-light-blue:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-light-blue.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(227, 241, 254, 0.5); -} - -.btn-outline-yellow { - color: #ffcc00; - border-color: #ffcc00; -} -.btn-outline-yellow:hover { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-outline-yellow:focus, .btn-outline-yellow.focus { - box-shadow: 0 0 0 0 rgba(255, 204, 0, 0.5); -} -.btn-outline-yellow.disabled, .btn-outline-yellow:disabled { - color: #ffcc00; - background-color: transparent; -} -.btn-outline-yellow:not(:disabled):not(.disabled):active, .btn-outline-yellow:not(:disabled):not(.disabled).active, .show > .btn-outline-yellow.dropdown-toggle { - color: #495057; - background-color: #ffcc00; - border-color: #ffcc00; -} -.btn-outline-yellow:not(:disabled):not(.disabled):active:focus, .btn-outline-yellow:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-yellow.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 204, 0, 0.5); -} - -.btn-outline-light-yellow { - color: #fffaf0; - border-color: #fffaf0; -} -.btn-outline-light-yellow:hover { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-outline-light-yellow:focus, .btn-outline-light-yellow.focus { - box-shadow: 0 0 0 0 rgba(255, 250, 240, 0.5); -} -.btn-outline-light-yellow.disabled, .btn-outline-light-yellow:disabled { - color: #fffaf0; - background-color: transparent; -} -.btn-outline-light-yellow:not(:disabled):not(.disabled):active, .btn-outline-light-yellow:not(:disabled):not(.disabled).active, .show > .btn-outline-light-yellow.dropdown-toggle { - color: #495057; - background-color: #fffaf0; - border-color: #fffaf0; -} -.btn-outline-light-yellow:not(:disabled):not(.disabled):active:focus, .btn-outline-light-yellow:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-light-yellow.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 250, 240, 0.5); -} - -.btn-outline-orange { - color: #ff8c00; - border-color: #ff8c00; -} -.btn-outline-orange:hover { - color: #495057; - background-color: #ff8c00; - border-color: #ff8c00; -} -.btn-outline-orange:focus, .btn-outline-orange.focus { - box-shadow: 0 0 0 0 rgba(255, 140, 0, 0.5); -} -.btn-outline-orange.disabled, .btn-outline-orange:disabled { - color: #ff8c00; - background-color: transparent; -} -.btn-outline-orange:not(:disabled):not(.disabled):active, .btn-outline-orange:not(:disabled):not(.disabled).active, .show > .btn-outline-orange.dropdown-toggle { - color: #495057; - background-color: #ff8c00; - border-color: #ff8c00; -} -.btn-outline-orange:not(:disabled):not(.disabled):active:focus, .btn-outline-orange:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-orange.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 140, 0, 0.5); -} - -.btn-outline-light-orange { - color: #ffe4b5; - border-color: #ffe4b5; -} -.btn-outline-light-orange:hover { - color: #495057; - background-color: #ffe4b5; - border-color: #ffe4b5; -} -.btn-outline-light-orange:focus, .btn-outline-light-orange.focus { - box-shadow: 0 0 0 0 rgba(255, 228, 181, 0.5); -} -.btn-outline-light-orange.disabled, .btn-outline-light-orange:disabled { - color: #ffe4b5; - background-color: transparent; -} -.btn-outline-light-orange:not(:disabled):not(.disabled):active, .btn-outline-light-orange:not(:disabled):not(.disabled).active, .show > .btn-outline-light-orange.dropdown-toggle { - color: #495057; - background-color: #ffe4b5; - border-color: #ffe4b5; -} -.btn-outline-light-orange:not(:disabled):not(.disabled):active:focus, .btn-outline-light-orange:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-light-orange.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 228, 181, 0.5); -} - -.btn-outline-red { - color: #ff3939; - border-color: #ff3939; -} -.btn-outline-red:hover { - color: #fff; - background-color: #ff3939; - border-color: #ff3939; -} -.btn-outline-red:focus, .btn-outline-red.focus { - box-shadow: 0 0 0 0 rgba(255, 57, 57, 0.5); -} -.btn-outline-red.disabled, .btn-outline-red:disabled { - color: #ff3939; - background-color: transparent; -} -.btn-outline-red:not(:disabled):not(.disabled):active, .btn-outline-red:not(:disabled):not(.disabled).active, .show > .btn-outline-red.dropdown-toggle { - color: #fff; - background-color: #ff3939; - border-color: #ff3939; -} -.btn-outline-red:not(:disabled):not(.disabled):active:focus, .btn-outline-red:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-red.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 57, 57, 0.5); -} - -.btn-outline-light-red { - color: #ffe4e1; - border-color: #ffe4e1; -} -.btn-outline-light-red:hover { - color: #495057; - background-color: #ffe4e1; - border-color: #ffe4e1; -} -.btn-outline-light-red:focus, .btn-outline-light-red.focus { - box-shadow: 0 0 0 0 rgba(255, 228, 225, 0.5); -} -.btn-outline-light-red.disabled, .btn-outline-light-red:disabled { - color: #ffe4e1; - background-color: transparent; -} -.btn-outline-light-red:not(:disabled):not(.disabled):active, .btn-outline-light-red:not(:disabled):not(.disabled).active, .show > .btn-outline-light-red.dropdown-toggle { - color: #495057; - background-color: #ffe4e1; - border-color: #ffe4e1; -} -.btn-outline-light-red:not(:disabled):not(.disabled):active:focus, .btn-outline-light-red:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-light-red.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(255, 228, 225, 0.5); -} - -.btn-outline-medium { - color: #d6dbdf; - border-color: #d6dbdf; -} -.btn-outline-medium:hover { - color: #495057; - background-color: #d6dbdf; - border-color: #d6dbdf; -} -.btn-outline-medium:focus, .btn-outline-medium.focus { - box-shadow: 0 0 0 0 rgba(214, 219, 223, 0.5); -} -.btn-outline-medium.disabled, .btn-outline-medium:disabled { - color: #d6dbdf; - background-color: transparent; -} -.btn-outline-medium:not(:disabled):not(.disabled):active, .btn-outline-medium:not(:disabled):not(.disabled).active, .show > .btn-outline-medium.dropdown-toggle { - color: #495057; - background-color: #d6dbdf; - border-color: #d6dbdf; -} -.btn-outline-medium:not(:disabled):not(.disabled):active:focus, .btn-outline-medium:not(:disabled):not(.disabled).active:focus, .show > .btn-outline-medium.dropdown-toggle:focus { - box-shadow: 0 0 0 0 rgba(214, 219, 223, 0.5); -} - -.btn-link { - font-weight: 400; - color: #ff8c00; - text-decoration: none; -} -.btn-link:hover { - color: #ff8c00; - text-decoration: underline; -} -.btn-link:focus, .btn-link.focus { - text-decoration: underline; - box-shadow: none; -} -.btn-link:disabled, .btn-link.disabled { - color: #d6dbdf; - pointer-events: none; -} - -.btn-lg, .btn-group-lg > .btn { - padding: 16px 32px; - font-size: 1.125rem; - line-height: 26px; - border-radius: 8px; -} - -.btn-sm, .btn-group-sm > .btn { - padding: 12px 32px; - font-size: 0.875rem; - line-height: 20px; - border-radius: 8px; -} - -.btn-block { - display: block; - width: 100%; -} -.btn-block + .btn-block { - margin-top: 24px; -} - -input[type=submit].btn-block, -input[type=reset].btn-block, -input[type=button].btn-block { - width: 100%; -} - -.fade { - transition: opacity 0.15s linear; -} -@media (prefers-reduced-motion: reduce) { - .fade { - transition: none; - } -} -.fade:not(.show) { - opacity: 0; -} - -.collapse:not(.show) { - display: none; -} - -.collapsing { - position: relative; - height: 0; - overflow: hidden; - transition: height 0.35s ease; -} -@media (prefers-reduced-motion: reduce) { - .collapsing { - transition: none; - } -} - -.dropup, -.dropright, -.dropdown, -.dropleft { - position: relative; -} - -.dropdown-toggle { - white-space: nowrap; -} -.dropdown-toggle::after { - display: inline-block; - margin-left: 0.255em; - vertical-align: 0.255em; - content: ""; - border-top: 0.3em solid; - border-right: 0.3em solid transparent; - border-bottom: 0; - border-left: 0.3em solid transparent; -} -.dropdown-toggle:empty::after { - margin-left: 0; -} - -.dropdown-menu { - position: absolute; - top: 100%; - left: 0; - z-index: 1000; - display: none; - float: left; - min-width: 10rem; - padding: 0.5rem 0; - margin: 0.125rem 0 0; - font-size: 1rem; - color: #212529; - text-align: left; - list-style: none; - background-color: #fff; - background-clip: padding-box; - border: 1px solid rgba(33, 37, 41, 0.15); - border-radius: 8px; -} - -.dropdown-menu-left { - right: auto; - left: 0; -} - -.dropdown-menu-right { - right: 0; - left: auto; -} - -@media (min-width: 400px) { - .dropdown-menu-xs-left { - right: auto; - left: 0; - } - - .dropdown-menu-xs-right { - right: 0; - left: auto; - } -} -@media (min-width: 616px) { - .dropdown-menu-sm-left { - right: auto; - left: 0; - } - - .dropdown-menu-sm-right { - right: 0; - left: auto; - } -} -@media (min-width: 768px) { - .dropdown-menu-md-left { - right: auto; - left: 0; - } - - .dropdown-menu-md-right { - right: 0; - left: auto; - } -} -@media (min-width: 980px) { - .dropdown-menu-lg-left { - right: auto; - left: 0; - } - - .dropdown-menu-lg-right { - right: 0; - left: auto; - } -} -@media (min-width: 1240px) { - .dropdown-menu-xl-left { - right: auto; - left: 0; - } - - .dropdown-menu-xl-right { - right: 0; - left: auto; - } -} -.dropup .dropdown-menu { - top: auto; - bottom: 100%; - margin-top: 0; - margin-bottom: 0.125rem; -} -.dropup .dropdown-toggle::after { - display: inline-block; - margin-left: 0.255em; - vertical-align: 0.255em; - content: ""; - border-top: 0; - border-right: 0.3em solid transparent; - border-bottom: 0.3em solid; - border-left: 0.3em solid transparent; -} -.dropup .dropdown-toggle:empty::after { - margin-left: 0; -} - -.dropright .dropdown-menu { - top: 0; - right: auto; - left: 100%; - margin-top: 0; - margin-left: 0.125rem; -} -.dropright .dropdown-toggle::after { - display: inline-block; - margin-left: 0.255em; - vertical-align: 0.255em; - content: ""; - border-top: 0.3em solid transparent; - border-right: 0; - border-bottom: 0.3em solid transparent; - border-left: 0.3em solid; -} -.dropright .dropdown-toggle:empty::after { - margin-left: 0; -} -.dropright .dropdown-toggle::after { - vertical-align: 0; -} - -.dropleft .dropdown-menu { - top: 0; - right: 100%; - left: auto; - margin-top: 0; - margin-right: 0.125rem; -} -.dropleft .dropdown-toggle::after { - display: inline-block; - margin-left: 0.255em; - vertical-align: 0.255em; - content: ""; -} -.dropleft .dropdown-toggle::after { - display: none; -} -.dropleft .dropdown-toggle::before { - display: inline-block; - margin-right: 0.255em; - vertical-align: 0.255em; - content: ""; - border-top: 0.3em solid transparent; - border-right: 0.3em solid; - border-bottom: 0.3em solid transparent; -} -.dropleft .dropdown-toggle:empty::after { - margin-left: 0; -} -.dropleft .dropdown-toggle::before { - vertical-align: 0; -} - -.dropdown-menu[x-placement^=top], .dropdown-menu[x-placement^=right], .dropdown-menu[x-placement^=bottom], .dropdown-menu[x-placement^=left] { - right: auto; - bottom: auto; -} - -.dropdown-divider { - height: 0; - margin: 4px 0; - overflow: hidden; - border-top: 1px solid #e9ecef; -} - -.dropdown-item { - display: block; - width: 100%; - padding: 0.25rem 1.5rem; - clear: both; - font-weight: 400; - color: #495057; - text-align: inherit; - white-space: nowrap; - background-color: transparent; - border: 0; -} -.dropdown-item:hover, .dropdown-item:focus { - color: #3d4349; - text-decoration: none; - background-color: #f1f6f9; -} -.dropdown-item.active, .dropdown-item:active { - color: #fff; - text-decoration: none; - background-color: #ffcc00; -} -.dropdown-item.disabled, .dropdown-item:disabled { - color: #6c757d; - pointer-events: none; - background-color: transparent; -} - -.dropdown-menu.show { - display: block; -} - -.dropdown-header { - display: block; - padding: 0.5rem 1.5rem; - margin-bottom: 0; - font-size: 0.875rem; - color: #6c757d; - white-space: nowrap; -} - -.dropdown-item-text { - display: block; - padding: 0.25rem 1.5rem; - color: #495057; -} - -.btn-group, -.btn-group-vertical { - position: relative; - display: inline-flex; - vertical-align: middle; -} -.btn-group > .btn, -.btn-group-vertical > .btn { - position: relative; - flex: 1 1 auto; -} -.btn-group > .btn:hover, -.btn-group-vertical > .btn:hover { - z-index: 1; -} -.btn-group > .btn:focus, .btn-group > .btn:active, .btn-group > .btn.active, -.btn-group-vertical > .btn:focus, -.btn-group-vertical > .btn:active, -.btn-group-vertical > .btn.active { - z-index: 1; -} - -.btn-toolbar { - display: flex; - flex-wrap: wrap; - justify-content: flex-start; -} -.btn-toolbar .input-group { - width: auto; -} - -.btn-group > .btn:not(:first-child), -.btn-group > .btn-group:not(:first-child) { - margin-left: -1px; -} -.btn-group > .btn:not(:last-child):not(.dropdown-toggle), -.btn-group > .btn-group:not(:last-child) > .btn { - border-top-right-radius: 0; - border-bottom-right-radius: 0; -} -.btn-group > .btn:not(:first-child), -.btn-group > .btn-group:not(:first-child) > .btn { - border-top-left-radius: 0; - border-bottom-left-radius: 0; -} - -.dropdown-toggle-split { - padding-right: 24px; - padding-left: 24px; -} -.dropdown-toggle-split::after, .dropup .dropdown-toggle-split::after, .dropright .dropdown-toggle-split::after { - margin-left: 0; -} -.dropleft .dropdown-toggle-split::before { - margin-right: 0; -} - -.btn-sm + .dropdown-toggle-split, .btn-group-sm > .btn + .dropdown-toggle-split { - padding-right: 24px; - padding-left: 24px; -} - -.btn-lg + .dropdown-toggle-split, .btn-group-lg > .btn + .dropdown-toggle-split { - padding-right: 24px; - padding-left: 24px; -} - -.btn-group-vertical { - flex-direction: column; - align-items: flex-start; - justify-content: center; -} -.btn-group-vertical > .btn, -.btn-group-vertical > .btn-group { - width: 100%; -} -.btn-group-vertical > .btn:not(:first-child), -.btn-group-vertical > .btn-group:not(:first-child) { - margin-top: -1px; -} -.btn-group-vertical > .btn:not(:last-child):not(.dropdown-toggle), -.btn-group-vertical > .btn-group:not(:last-child) > .btn { - border-bottom-right-radius: 0; - border-bottom-left-radius: 0; -} -.btn-group-vertical > .btn:not(:first-child), -.btn-group-vertical > .btn-group:not(:first-child) > .btn { - border-top-left-radius: 0; - border-top-right-radius: 0; -} - -.btn-group-toggle > .btn, -.btn-group-toggle > .btn-group > .btn { - margin-bottom: 0; -} -.btn-group-toggle > .btn input[type=radio], -.btn-group-toggle > .btn input[type=checkbox], -.btn-group-toggle > .btn-group > .btn input[type=radio], -.btn-group-toggle > .btn-group > .btn input[type=checkbox] { - position: absolute; - clip: rect(0, 0, 0, 0); - pointer-events: none; -} - -.input-group { - position: relative; - display: flex; - flex-wrap: wrap; - align-items: stretch; - width: 100%; -} -.input-group > .form-control, -.input-group > .form-control-plaintext, -.input-group > .custom-select, -.input-group > .custom-file { - position: relative; - flex: 1 1 0%; - min-width: 0; - margin-bottom: 0; -} -.input-group > .form-control + .form-control, -.input-group > .form-control + .custom-select, -.input-group > .form-control + .custom-file, -.input-group > .form-control-plaintext + .form-control, -.input-group > .form-control-plaintext + .custom-select, -.input-group > .form-control-plaintext + .custom-file, -.input-group > .custom-select + .form-control, -.input-group > .custom-select + .custom-select, -.input-group > .custom-select + .custom-file, -.input-group > .custom-file + .form-control, -.input-group > .custom-file + .custom-select, -.input-group > .custom-file + .custom-file { - margin-left: -1px; -} -.input-group > .form-control:focus, -.input-group > .custom-select:focus, -.input-group > .custom-file .custom-file-input:focus ~ .custom-file-label { - z-index: 3; -} -.input-group > .custom-file .custom-file-input:focus { - z-index: 4; -} -.input-group > .form-control:not(:last-child), -.input-group > .custom-select:not(:last-child) { - border-top-right-radius: 0; - border-bottom-right-radius: 0; -} -.input-group > .form-control:not(:first-child), -.input-group > .custom-select:not(:first-child) { - border-top-left-radius: 0; - border-bottom-left-radius: 0; -} -.input-group > .custom-file { - display: flex; - align-items: center; -} -.input-group > .custom-file:not(:last-child) .custom-file-label, .input-group > .custom-file:not(:last-child) .custom-file-label::after { - border-top-right-radius: 0; - border-bottom-right-radius: 0; -} -.input-group > .custom-file:not(:first-child) .custom-file-label { - border-top-left-radius: 0; - border-bottom-left-radius: 0; -} - -.input-group-prepend, -.input-group-append { - display: flex; -} -.input-group-prepend .btn, -.input-group-append .btn { - position: relative; - z-index: 2; -} -.input-group-prepend .btn:focus, -.input-group-append .btn:focus { - z-index: 3; -} -.input-group-prepend .btn + .btn, -.input-group-prepend .btn + .input-group-text, -.input-group-prepend .input-group-text + .input-group-text, -.input-group-prepend .input-group-text + .btn, -.input-group-append .btn + .btn, -.input-group-append .btn + .input-group-text, -.input-group-append .input-group-text + .input-group-text, -.input-group-append .input-group-text + .btn { - margin-left: -1px; -} - -.input-group-prepend { - margin-right: -1px; -} - -.input-group-append { - margin-left: -1px; -} - -.input-group-text { - display: flex; - align-items: center; - padding: 0.375rem 0.75rem; - margin-bottom: 0; - font-size: 1rem; - font-weight: 400; - line-height: 1.5; - color: #6c757d; - text-align: center; - white-space: nowrap; - background-color: #e9ecef; - border: 1px solid #ced4da; - border-radius: 8px; -} -.input-group-text input[type=radio], -.input-group-text input[type=checkbox] { - margin-top: 0; -} - -.input-group-lg > .form-control:not(textarea), -.input-group-lg > .custom-select { - height: calc(1.5em + 1rem + 2px); -} - -.input-group-lg > .form-control, -.input-group-lg > .custom-select, -.input-group-lg > .input-group-prepend > .input-group-text, -.input-group-lg > .input-group-append > .input-group-text, -.input-group-lg > .input-group-prepend > .btn, -.input-group-lg > .input-group-append > .btn { - padding: 0.5rem 1rem; - font-size: 1.125rem; - line-height: 1.5; - border-radius: 8px; -} - -.input-group-sm > .form-control:not(textarea), -.input-group-sm > .custom-select { - height: calc(1.5em + 0.5rem + 2px); -} - -.input-group-sm > .form-control, -.input-group-sm > .custom-select, -.input-group-sm > .input-group-prepend > .input-group-text, -.input-group-sm > .input-group-append > .input-group-text, -.input-group-sm > .input-group-prepend > .btn, -.input-group-sm > .input-group-append > .btn { - padding: 0.25rem 0.5rem; - font-size: 0.875rem; - line-height: 1.5; - border-radius: 8px; -} - -.input-group-lg > .custom-select, -.input-group-sm > .custom-select { - padding-right: 1.75rem; -} - -.input-group > .input-group-prepend > .btn, -.input-group > .input-group-prepend > .input-group-text, -.input-group > .input-group-append:not(:last-child) > .btn, -.input-group > .input-group-append:not(:last-child) > .input-group-text, -.input-group > .input-group-append:last-child > .btn:not(:last-child):not(.dropdown-toggle), -.input-group > .input-group-append:last-child > .input-group-text:not(:last-child) { - border-top-right-radius: 0; - border-bottom-right-radius: 0; -} - -.input-group > .input-group-append > .btn, -.input-group > .input-group-append > .input-group-text, -.input-group > .input-group-prepend:not(:first-child) > .btn, -.input-group > .input-group-prepend:not(:first-child) > .input-group-text, -.input-group > .input-group-prepend:first-child > .btn:not(:first-child), -.input-group > .input-group-prepend:first-child > .input-group-text:not(:first-child) { - border-top-left-radius: 0; - border-bottom-left-radius: 0; -} - -.custom-control { - position: relative; - display: block; - min-height: 1.5rem; - padding-left: 1.5rem; -} - -.custom-control-inline { - display: inline-flex; - margin-right: 1rem; -} - -.custom-control-input { - position: absolute; - left: 0; - z-index: -1; - width: 1rem; - height: 1.25rem; - opacity: 0; -} -.custom-control-input:checked ~ .custom-control-label::before { - color: #fff; - border-color: #ffcc00; - background-color: #ffcc00; -} -.custom-control-input:focus ~ .custom-control-label::before { - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.custom-control-input:focus:not(:checked) ~ .custom-control-label::before { - border-color: #ffe680; -} -.custom-control-input:not(:disabled):active ~ .custom-control-label::before { - color: #fff; - background-color: #fff0b3; - border-color: #fff0b3; -} -.custom-control-input[disabled] ~ .custom-control-label, .custom-control-input:disabled ~ .custom-control-label { - color: #6c757d; -} -.custom-control-input[disabled] ~ .custom-control-label::before, .custom-control-input:disabled ~ .custom-control-label::before { - background-color: #e9ecef; -} - -.custom-control-label { - position: relative; - margin-bottom: 0; - vertical-align: top; -} -.custom-control-label::before { - position: absolute; - top: 0.25rem; - left: -1.5rem; - display: block; - width: 1rem; - height: 1rem; - pointer-events: none; - content: ""; - background-color: #fff; - border: #d6dbdf solid 1px; -} -.custom-control-label::after { - position: absolute; - top: 0.25rem; - left: -1.5rem; - display: block; - width: 1rem; - height: 1rem; - content: ""; - background: no-repeat 50%/50% 50%; -} - -.custom-checkbox .custom-control-label::before { - border-radius: 8px; -} -.custom-checkbox .custom-control-input:checked ~ .custom-control-label::after { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath fill='%23fff' d='M6.564.75l-3.59 3.612-1.538-1.55L0 4.26l2.974 2.99L8 2.193z'/%3e%3c/svg%3e"); -} -.custom-checkbox .custom-control-input:indeterminate ~ .custom-control-label::before { - border-color: #ffcc00; - background-color: #ffcc00; -} -.custom-checkbox .custom-control-input:indeterminate ~ .custom-control-label::after { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='4' viewBox='0 0 4 4'%3e%3cpath stroke='%23fff' d='M0 2h4'/%3e%3c/svg%3e"); -} -.custom-checkbox .custom-control-input:disabled:checked ~ .custom-control-label::before { - background-color: rgba(255, 204, 0, 0.5); -} -.custom-checkbox .custom-control-input:disabled:indeterminate ~ .custom-control-label::before { - background-color: rgba(255, 204, 0, 0.5); -} - -.custom-radio .custom-control-label::before { - border-radius: 50%; -} -.custom-radio .custom-control-input:checked ~ .custom-control-label::after { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' viewBox='-4 -4 8 8'%3e%3ccircle r='3' fill='%23fff'/%3e%3c/svg%3e"); -} -.custom-radio .custom-control-input:disabled:checked ~ .custom-control-label::before { - background-color: rgba(255, 204, 0, 0.5); -} - -.custom-switch { - padding-left: 2.25rem; -} -.custom-switch .custom-control-label::before { - left: -2.25rem; - width: 1.75rem; - pointer-events: all; - border-radius: 0.5rem; -} -.custom-switch .custom-control-label::after { - top: calc(0.25rem + 2px); - left: calc(-2.25rem + 2px); - width: calc(1rem - 4px); - height: calc(1rem - 4px); - background-color: #d6dbdf; - border-radius: 0.5rem; - transition: transform 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; -} -@media (prefers-reduced-motion: reduce) { - .custom-switch .custom-control-label::after { - transition: none; - } -} -.custom-switch .custom-control-input:checked ~ .custom-control-label::after { - background-color: #fff; - transform: translateX(0.75rem); -} -.custom-switch .custom-control-input:disabled:checked ~ .custom-control-label::before { - background-color: rgba(255, 204, 0, 0.5); -} - -.custom-select { - display: inline-block; - width: 100%; - height: calc(1.5em + 0.75rem + 2px); - padding: 0.375rem 1.75rem 0.375rem 0.75rem; - font-size: 1rem; - font-weight: 400; - line-height: 1.5; - color: #6c757d; - vertical-align: middle; - background: #fff url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5' viewBox='0 0 4 5'%3e%3cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3e%3c/svg%3e") no-repeat right 0.75rem center/8px 10px; - border: 1px solid #ced4da; - border-radius: 8px; - -webkit-appearance: none; - -moz-appearance: none; - appearance: none; -} -.custom-select:focus { - border-color: #ffe680; - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.custom-select:focus::-ms-value { - color: #6c757d; - background-color: #fff; -} -.custom-select[multiple], .custom-select[size]:not([size="1"]) { - height: auto; - padding-right: 0.75rem; - background-image: none; -} -.custom-select:disabled { - color: #6c757d; - background-color: #e9ecef; -} -.custom-select::-ms-expand { - display: none; -} -.custom-select:-moz-focusring { - color: transparent; - text-shadow: 0 0 0 #6c757d; -} - -.custom-select-sm { - height: calc(1.5em + 0.5rem + 2px); - padding-top: 0.25rem; - padding-bottom: 0.25rem; - padding-left: 0.5rem; - font-size: 0.875rem; -} - -.custom-select-lg { - height: calc(1.5em + 1rem + 2px); - padding-top: 0.5rem; - padding-bottom: 0.5rem; - padding-left: 1rem; - font-size: 1.125rem; -} - -.custom-file { - position: relative; - display: inline-block; - width: 100%; - height: calc(1.5em + 0.75rem + 2px); - margin-bottom: 0; -} - -.custom-file-input { - position: relative; - z-index: 2; - width: 100%; - height: calc(1.5em + 0.75rem + 2px); - margin: 0; - opacity: 0; -} -.custom-file-input:focus ~ .custom-file-label { - border-color: #ffe680; - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.custom-file-input[disabled] ~ .custom-file-label, .custom-file-input:disabled ~ .custom-file-label { - background-color: #e9ecef; -} -.custom-file-input:lang(en) ~ .custom-file-label::after { - content: "Browse"; -} -.custom-file-input ~ .custom-file-label[data-browse]::after { - content: attr(data-browse); -} - -.custom-file-label { - position: absolute; - top: 0; - right: 0; - left: 0; - z-index: 1; - height: calc(1.5em + 0.75rem + 2px); - padding: 0.375rem 0.75rem; - font-weight: 400; - line-height: 1.5; - color: #6c757d; - background-color: #fff; - border: 1px solid #ced4da; - border-radius: 8px; -} -.custom-file-label::after { - position: absolute; - top: 0; - right: 0; - bottom: 0; - z-index: 3; - display: block; - height: calc(1.5em + 0.75rem); - padding: 0.375rem 0.75rem; - line-height: 1.5; - color: #6c757d; - content: "Browse"; - background-color: #e9ecef; - border-left: inherit; - border-radius: 0 8px 8px 0; -} - -.custom-range { - width: 100%; - height: 1.4rem; - padding: 0; - background-color: transparent; - -webkit-appearance: none; - -moz-appearance: none; - appearance: none; -} -.custom-range:focus { - outline: none; -} -.custom-range:focus::-webkit-slider-thumb { - box-shadow: 0 0 0 1px #fff, 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.custom-range:focus::-moz-range-thumb { - box-shadow: 0 0 0 1px #fff, 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.custom-range:focus::-ms-thumb { - box-shadow: 0 0 0 1px #fff, 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} -.custom-range::-moz-focus-outer { - border: 0; -} -.custom-range::-webkit-slider-thumb { - width: 1rem; - height: 1rem; - margin-top: -0.25rem; - background-color: #ffcc00; - border: 0; - border-radius: 1rem; - -webkit-transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; - transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; - -webkit-appearance: none; - appearance: none; -} -@media (prefers-reduced-motion: reduce) { - .custom-range::-webkit-slider-thumb { - -webkit-transition: none; - transition: none; - } -} -.custom-range::-webkit-slider-thumb:active { - background-color: #fff0b3; -} -.custom-range::-webkit-slider-runnable-track { - width: 100%; - height: 0.5rem; - color: transparent; - cursor: pointer; - background-color: #dee2e6; - border-color: transparent; - border-radius: 1rem; -} -.custom-range::-moz-range-thumb { - width: 1rem; - height: 1rem; - background-color: #ffcc00; - border: 0; - border-radius: 1rem; - -moz-transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; - transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; - -moz-appearance: none; - appearance: none; -} -@media (prefers-reduced-motion: reduce) { - .custom-range::-moz-range-thumb { - -moz-transition: none; - transition: none; - } -} -.custom-range::-moz-range-thumb:active { - background-color: #fff0b3; -} -.custom-range::-moz-range-track { - width: 100%; - height: 0.5rem; - color: transparent; - cursor: pointer; - background-color: #dee2e6; - border-color: transparent; - border-radius: 1rem; -} -.custom-range::-ms-thumb { - width: 1rem; - height: 1rem; - margin-top: 0; - margin-right: 0.2rem; - margin-left: 0.2rem; - background-color: #ffcc00; - border: 0; - border-radius: 1rem; - -ms-transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; - transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; - appearance: none; -} -@media (prefers-reduced-motion: reduce) { - .custom-range::-ms-thumb { - -ms-transition: none; - transition: none; - } -} -.custom-range::-ms-thumb:active { - background-color: #fff0b3; -} -.custom-range::-ms-track { - width: 100%; - height: 0.5rem; - color: transparent; - cursor: pointer; - background-color: transparent; - border-color: transparent; - border-width: 0.5rem; -} -.custom-range::-ms-fill-lower { - background-color: #dee2e6; - border-radius: 1rem; -} -.custom-range::-ms-fill-upper { - margin-right: 15px; - background-color: #dee2e6; - border-radius: 1rem; -} -.custom-range:disabled::-webkit-slider-thumb { - background-color: #d6dbdf; -} -.custom-range:disabled::-webkit-slider-runnable-track { - cursor: default; -} -.custom-range:disabled::-moz-range-thumb { - background-color: #d6dbdf; -} -.custom-range:disabled::-moz-range-track { - cursor: default; -} -.custom-range:disabled::-ms-thumb { - background-color: #d6dbdf; -} - -.custom-control-label::before, -.custom-file-label, -.custom-select { - transition: background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; -} -@media (prefers-reduced-motion: reduce) { - .custom-control-label::before, -.custom-file-label, -.custom-select { - transition: none; - } -} - -.nav { - display: flex; - flex-wrap: wrap; - padding-left: 0; - margin-bottom: 0; - list-style: none; -} - -.nav-link { - display: block; - padding: 0 0; -} -.nav-link:hover, .nav-link:focus { - text-decoration: none; -} -.nav-link.disabled { - color: #d6dbdf; - pointer-events: none; - cursor: default; -} - -.nav-tabs { - border-bottom: 1px solid #6c757d; -} -.nav-tabs .nav-item { - margin-bottom: -1px; -} -.nav-tabs .nav-link { - border: 1px solid transparent; - border-top-left-radius: 8px; - border-top-right-radius: 8px; -} -.nav-tabs .nav-link:hover, .nav-tabs .nav-link:focus { - border-color: transparent; -} -.nav-tabs .nav-link.disabled { - color: #d6dbdf; - background-color: transparent; - border-color: transparent; -} -.nav-tabs .nav-link.active, -.nav-tabs .nav-item.show .nav-link { - color: #257af4; - background-color: #fff; - border-color: #6c757d; -} -.nav-tabs .dropdown-menu { - margin-top: -1px; - border-top-left-radius: 0; - border-top-right-radius: 0; -} - -.nav-pills .nav-link { - border-radius: 8px; -} -.nav-pills .nav-link.active, -.nav-pills .show > .nav-link { - color: #fff; - background-color: #ffcc00; -} - -.nav-fill .nav-item { - flex: 1 1 auto; - text-align: center; -} - -.nav-justified .nav-item { - flex-basis: 0; - flex-grow: 1; - text-align: center; -} - -.tab-content > .tab-pane { - display: none; -} -.tab-content > .active { - display: block; -} - -.navbar { - position: relative; - display: flex; - flex-wrap: wrap; - align-items: center; - justify-content: space-between; - padding: 24px 0; -} -.navbar .container, -.navbar .container-fluid, -.navbar .container-xs, -.navbar .container-sm, -.navbar .container-md, -.navbar .container-lg, -.navbar .container-xl { - display: flex; - flex-wrap: wrap; - align-items: center; - justify-content: space-between; -} -.navbar-brand { - display: inline-block; - padding-top: -0.09375rem; - padding-bottom: -0.09375rem; - margin-right: 0; - font-size: 1.125rem; - line-height: inherit; - white-space: nowrap; -} -.navbar-brand:hover, .navbar-brand:focus { - text-decoration: none; -} - -.navbar-nav { - display: flex; - flex-direction: column; - padding-left: 0; - margin-bottom: 0; - list-style: none; -} -.navbar-nav .nav-link { - padding-right: 0; - padding-left: 0; -} -.navbar-nav .dropdown-menu { - position: static; - float: none; -} - -.navbar-text { - display: inline-block; - padding-top: 0; - padding-bottom: 0; -} - -.navbar-collapse { - flex-basis: 100%; - flex-grow: 1; - align-items: center; -} - -.navbar-toggler { - padding: 0.25rem 0.75rem; - font-size: 1.125rem; - line-height: 1; - background-color: transparent; - border: 1px solid transparent; - border-radius: 8px; -} -.navbar-toggler:hover, .navbar-toggler:focus { - text-decoration: none; -} - -.navbar-toggler-icon { - display: inline-block; - width: 1.5em; - height: 1.5em; - vertical-align: middle; - content: ""; - background: no-repeat center center; - background-size: 100% 100%; -} - -@media (max-width: 399.98px) { - .navbar-expand-xs > .container, -.navbar-expand-xs > .container-fluid, -.navbar-expand-xs > .container-xs, -.navbar-expand-xs > .container-sm, -.navbar-expand-xs > .container-md, -.navbar-expand-xs > .container-lg, -.navbar-expand-xs > .container-xl { - padding-right: 0; - padding-left: 0; - } -} -@media (min-width: 400px) { - .navbar-expand-xs { - flex-flow: row nowrap; - justify-content: flex-start; - } - .navbar-expand-xs .navbar-nav { - flex-direction: row; - } - .navbar-expand-xs .navbar-nav .dropdown-menu { - position: absolute; - } - .navbar-expand-xs .navbar-nav .nav-link { - padding-right: 0.5rem; - padding-left: 0.5rem; - } - .navbar-expand-xs > .container, -.navbar-expand-xs > .container-fluid, -.navbar-expand-xs > .container-xs, -.navbar-expand-xs > .container-sm, -.navbar-expand-xs > .container-md, -.navbar-expand-xs > .container-lg, -.navbar-expand-xs > .container-xl { - flex-wrap: nowrap; - } - .navbar-expand-xs .navbar-collapse { - display: flex !important; - flex-basis: auto; - } - .navbar-expand-xs .navbar-toggler { - display: none; - } -} -@media (max-width: 615.98px) { - .navbar-expand-sm > .container, -.navbar-expand-sm > .container-fluid, -.navbar-expand-sm > .container-xs, -.navbar-expand-sm > .container-sm, -.navbar-expand-sm > .container-md, -.navbar-expand-sm > .container-lg, -.navbar-expand-sm > .container-xl { - padding-right: 0; - padding-left: 0; - } -} -@media (min-width: 616px) { - .navbar-expand-sm { - flex-flow: row nowrap; - justify-content: flex-start; - } - .navbar-expand-sm .navbar-nav { - flex-direction: row; - } - .navbar-expand-sm .navbar-nav .dropdown-menu { - position: absolute; - } - .navbar-expand-sm .navbar-nav .nav-link { - padding-right: 0.5rem; - padding-left: 0.5rem; - } - .navbar-expand-sm > .container, -.navbar-expand-sm > .container-fluid, -.navbar-expand-sm > .container-xs, -.navbar-expand-sm > .container-sm, -.navbar-expand-sm > .container-md, -.navbar-expand-sm > .container-lg, -.navbar-expand-sm > .container-xl { - flex-wrap: nowrap; - } - .navbar-expand-sm .navbar-collapse { - display: flex !important; - flex-basis: auto; - } - .navbar-expand-sm .navbar-toggler { - display: none; - } -} -@media (max-width: 767.98px) { - .navbar-expand-md > .container, -.navbar-expand-md > .container-fluid, -.navbar-expand-md > .container-xs, -.navbar-expand-md > .container-sm, -.navbar-expand-md > .container-md, -.navbar-expand-md > .container-lg, -.navbar-expand-md > .container-xl { - padding-right: 0; - padding-left: 0; - } -} -@media (min-width: 768px) { - .navbar-expand-md { - flex-flow: row nowrap; - justify-content: flex-start; - } - .navbar-expand-md .navbar-nav { - flex-direction: row; - } - .navbar-expand-md .navbar-nav .dropdown-menu { - position: absolute; - } - .navbar-expand-md .navbar-nav .nav-link { - padding-right: 0.5rem; - padding-left: 0.5rem; - } - .navbar-expand-md > .container, -.navbar-expand-md > .container-fluid, -.navbar-expand-md > .container-xs, -.navbar-expand-md > .container-sm, -.navbar-expand-md > .container-md, -.navbar-expand-md > .container-lg, -.navbar-expand-md > .container-xl { - flex-wrap: nowrap; - } - .navbar-expand-md .navbar-collapse { - display: flex !important; - flex-basis: auto; - } - .navbar-expand-md .navbar-toggler { - display: none; - } -} -@media (max-width: 979.98px) { - .navbar-expand-lg > .container, -.navbar-expand-lg > .container-fluid, -.navbar-expand-lg > .container-xs, -.navbar-expand-lg > .container-sm, -.navbar-expand-lg > .container-md, -.navbar-expand-lg > .container-lg, -.navbar-expand-lg > .container-xl { - padding-right: 0; - padding-left: 0; - } -} -@media (min-width: 980px) { - .navbar-expand-lg { - flex-flow: row nowrap; - justify-content: flex-start; - } - .navbar-expand-lg .navbar-nav { - flex-direction: row; - } - .navbar-expand-lg .navbar-nav .dropdown-menu { - position: absolute; - } - .navbar-expand-lg .navbar-nav .nav-link { - padding-right: 0.5rem; - padding-left: 0.5rem; - } - .navbar-expand-lg > .container, -.navbar-expand-lg > .container-fluid, -.navbar-expand-lg > .container-xs, -.navbar-expand-lg > .container-sm, -.navbar-expand-lg > .container-md, -.navbar-expand-lg > .container-lg, -.navbar-expand-lg > .container-xl { - flex-wrap: nowrap; - } - .navbar-expand-lg .navbar-collapse { - display: flex !important; - flex-basis: auto; - } - .navbar-expand-lg .navbar-toggler { - display: none; - } -} -@media (max-width: 1239.98px) { - .navbar-expand-xl > .container, -.navbar-expand-xl > .container-fluid, -.navbar-expand-xl > .container-xs, -.navbar-expand-xl > .container-sm, -.navbar-expand-xl > .container-md, -.navbar-expand-xl > .container-lg, -.navbar-expand-xl > .container-xl { - padding-right: 0; - padding-left: 0; - } -} -@media (min-width: 1240px) { - .navbar-expand-xl { - flex-flow: row nowrap; - justify-content: flex-start; - } - .navbar-expand-xl .navbar-nav { - flex-direction: row; - } - .navbar-expand-xl .navbar-nav .dropdown-menu { - position: absolute; - } - .navbar-expand-xl .navbar-nav .nav-link { - padding-right: 0.5rem; - padding-left: 0.5rem; - } - .navbar-expand-xl > .container, -.navbar-expand-xl > .container-fluid, -.navbar-expand-xl > .container-xs, -.navbar-expand-xl > .container-sm, -.navbar-expand-xl > .container-md, -.navbar-expand-xl > .container-lg, -.navbar-expand-xl > .container-xl { - flex-wrap: nowrap; - } - .navbar-expand-xl .navbar-collapse { - display: flex !important; - flex-basis: auto; - } - .navbar-expand-xl .navbar-toggler { - display: none; - } -} -.navbar-expand { - flex-flow: row nowrap; - justify-content: flex-start; -} -.navbar-expand > .container, -.navbar-expand > .container-fluid, -.navbar-expand > .container-xs, -.navbar-expand > .container-sm, -.navbar-expand > .container-md, -.navbar-expand > .container-lg, -.navbar-expand > .container-xl { - padding-right: 0; - padding-left: 0; -} -.navbar-expand .navbar-nav { - flex-direction: row; -} -.navbar-expand .navbar-nav .dropdown-menu { - position: absolute; -} -.navbar-expand .navbar-nav .nav-link { - padding-right: 0.5rem; - padding-left: 0.5rem; -} -.navbar-expand > .container, -.navbar-expand > .container-fluid, -.navbar-expand > .container-xs, -.navbar-expand > .container-sm, -.navbar-expand > .container-md, -.navbar-expand > .container-lg, -.navbar-expand > .container-xl { - flex-wrap: nowrap; -} -.navbar-expand .navbar-collapse { - display: flex !important; - flex-basis: auto; -} -.navbar-expand .navbar-toggler { - display: none; -} - -.navbar-light .navbar-brand { - color: rgba(33, 37, 41, 0.9); -} -.navbar-light .navbar-brand:hover, .navbar-light .navbar-brand:focus { - color: rgba(33, 37, 41, 0.9); -} -.navbar-light .navbar-nav .nav-link { - color: rgba(33, 37, 41, 0.5); -} -.navbar-light .navbar-nav .nav-link:hover, .navbar-light .navbar-nav .nav-link:focus { - color: rgba(33, 37, 41, 0.7); -} -.navbar-light .navbar-nav .nav-link.disabled { - color: rgba(33, 37, 41, 0.3); -} -.navbar-light .navbar-nav .show > .nav-link, -.navbar-light .navbar-nav .active > .nav-link, -.navbar-light .navbar-nav .nav-link.show, -.navbar-light .navbar-nav .nav-link.active { - color: rgba(33, 37, 41, 0.9); -} -.navbar-light .navbar-toggler { - color: rgba(33, 37, 41, 0.5); - border-color: rgba(33, 37, 41, 0.1); -} -.navbar-light .navbar-toggler-icon { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30' viewBox='0 0 30 30'%3e%3cpath stroke='rgba(33, 37, 41, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3e%3c/svg%3e"); -} -.navbar-light .navbar-text { - color: rgba(33, 37, 41, 0.5); -} -.navbar-light .navbar-text a { - color: rgba(33, 37, 41, 0.9); -} -.navbar-light .navbar-text a:hover, .navbar-light .navbar-text a:focus { - color: rgba(33, 37, 41, 0.9); -} - -.navbar-dark .navbar-brand { - color: #fff; -} -.navbar-dark .navbar-brand:hover, .navbar-dark .navbar-brand:focus { - color: #fff; -} -.navbar-dark .navbar-nav .nav-link { - color: rgba(255, 255, 255, 0.5); -} -.navbar-dark .navbar-nav .nav-link:hover, .navbar-dark .navbar-nav .nav-link:focus { - color: rgba(255, 255, 255, 0.75); -} -.navbar-dark .navbar-nav .nav-link.disabled { - color: rgba(255, 255, 255, 0.25); -} -.navbar-dark .navbar-nav .show > .nav-link, -.navbar-dark .navbar-nav .active > .nav-link, -.navbar-dark .navbar-nav .nav-link.show, -.navbar-dark .navbar-nav .nav-link.active { - color: #fff; -} -.navbar-dark .navbar-toggler { - color: rgba(255, 255, 255, 0.5); - border-color: rgba(255, 255, 255, 0.1); -} -.navbar-dark .navbar-toggler-icon { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30' viewBox='0 0 30 30'%3e%3cpath stroke='rgba(255, 255, 255, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3e%3c/svg%3e"); -} -.navbar-dark .navbar-text { - color: rgba(255, 255, 255, 0.5); -} -.navbar-dark .navbar-text a { - color: #fff; -} -.navbar-dark .navbar-text a:hover, .navbar-dark .navbar-text a:focus { - color: #fff; -} - -.card { - position: relative; - display: flex; - flex-direction: column; - min-width: 0; - word-wrap: break-word; - background-color: #fff; - background-clip: border-box; - border: 1px solid #d6dbdf; - border-radius: 8px; -} -.card > hr { - margin-right: 0; - margin-left: 0; -} -.card > .list-group:first-child .list-group-item:first-child { - border-top-left-radius: 8px; - border-top-right-radius: 8px; -} -.card > .list-group:last-child .list-group-item:last-child { - border-bottom-right-radius: 8px; - border-bottom-left-radius: 8px; -} - -.card-body { - flex: 1 1 auto; - min-height: 1px; - padding: 24px; -} - -.card-title { - margin-bottom: 24px; -} - -.card-subtitle { - margin-top: -12px; - margin-bottom: 0; -} - -.card-text:last-child { - margin-bottom: 0; -} - -.card-link:hover { - text-decoration: none; -} -.card-link + .card-link { - margin-left: 24px; -} - -.card-header { - padding: 24px 24px; - margin-bottom: 0; - background-color: #f1f6f9; - border-bottom: 1px solid #d6dbdf; -} -.card-header:first-child { - border-radius: subtract(8px, 1px) subtract(8px, 1px) 0 0; -} -.card-header + .list-group .list-group-item:first-child { - border-top: 0; -} - -.card-footer { - padding: 24px 24px; - background-color: #f1f6f9; - border-top: 1px solid #d6dbdf; -} -.card-footer:last-child { - border-radius: 0 0 subtract(8px, 1px) subtract(8px, 1px); -} - -.card-header-tabs { - margin-right: -12px; - margin-bottom: -24px; - margin-left: -12px; - border-bottom: 0; -} - -.card-header-pills { - margin-right: -12px; - margin-left: -12px; -} - -.card-img-overlay { - position: absolute; - top: 0; - right: 0; - bottom: 0; - left: 0; - padding: 24px; -} - -.card-img, -.card-img-top, -.card-img-bottom { - flex-shrink: 0; - width: 100%; -} - -.card-img, -.card-img-top { - border-top-left-radius: subtract(8px, 1px); - border-top-right-radius: subtract(8px, 1px); -} - -.card-img, -.card-img-bottom { - border-bottom-right-radius: subtract(8px, 1px); - border-bottom-left-radius: subtract(8px, 1px); -} - -.card-deck .card { - margin-bottom: 20px; -} -@media (min-width: 616px) { - .card-deck { - display: flex; - flex-flow: row wrap; - margin-right: -20px; - margin-left: -20px; - } - .card-deck .card { - flex: 1 0 0%; - margin-right: 20px; - margin-bottom: 0; - margin-left: 20px; - } -} - -.card-group > .card { - margin-bottom: 20px; -} -@media (min-width: 616px) { - .card-group { - display: flex; - flex-flow: row wrap; - } - .card-group > .card { - flex: 1 0 0%; - margin-bottom: 0; - } - .card-group > .card + .card { - margin-left: 0; - border-left: 0; - } - .card-group > .card:not(:last-child) { - border-top-right-radius: 0; - border-bottom-right-radius: 0; - } - .card-group > .card:not(:last-child) .card-img-top, -.card-group > .card:not(:last-child) .card-header { - border-top-right-radius: 0; - } - .card-group > .card:not(:last-child) .card-img-bottom, -.card-group > .card:not(:last-child) .card-footer { - border-bottom-right-radius: 0; - } - .card-group > .card:not(:first-child) { - border-top-left-radius: 0; - border-bottom-left-radius: 0; - } - .card-group > .card:not(:first-child) .card-img-top, -.card-group > .card:not(:first-child) .card-header { - border-top-left-radius: 0; - } - .card-group > .card:not(:first-child) .card-img-bottom, -.card-group > .card:not(:first-child) .card-footer { - border-bottom-left-radius: 0; - } -} - -.card-columns .card { - margin-bottom: 40px; -} -@media (min-width: 616px) { - .card-columns { - -moz-column-count: 3; - column-count: 3; - -moz-column-gap: 40px; - column-gap: 40px; - orphans: 1; - widows: 1; - } - .card-columns .card { - display: inline-block; - width: 100%; - } -} - -.accordion > .card { - overflow: hidden; -} -.accordion > .card:not(:last-of-type) { - border-bottom: 0; - border-bottom-right-radius: 0; - border-bottom-left-radius: 0; -} -.accordion > .card:not(:first-of-type) { - border-top-left-radius: 0; - border-top-right-radius: 0; -} -.accordion > .card > .card-header { - border-radius: 0; - margin-bottom: -1px; -} - -.breadcrumb { - display: flex; - flex-wrap: wrap; - padding: 0.75rem 1rem; - margin-bottom: 1rem; - list-style: none; - background-color: #e9ecef; - border-radius: 8px; -} - -.breadcrumb-item + .breadcrumb-item { - padding-left: 0.5rem; -} -.breadcrumb-item + .breadcrumb-item::before { - display: inline-block; - padding-right: 0.5rem; - color: #6c757d; - content: "/"; -} -.breadcrumb-item + .breadcrumb-item:hover::before { - text-decoration: underline; -} -.breadcrumb-item + .breadcrumb-item:hover::before { - text-decoration: none; -} -.breadcrumb-item.active { - color: #6c757d; -} - -.pagination { - display: flex; - padding-left: 0; - list-style: none; - border-radius: 8px; -} - -.page-link { - position: relative; - display: block; - padding: 0.5rem 0.75rem; - margin-left: -1px; - line-height: 1.25; - color: #ff8c00; - background-color: #fff; - border: 1px solid #dee2e6; -} -.page-link:hover { - z-index: 2; - color: #ff8c00; - text-decoration: none; - background-color: #e9ecef; - border-color: #dee2e6; -} -.page-link:focus { - z-index: 3; - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.25); -} - -.page-item:first-child .page-link { - margin-left: 0; - border-top-left-radius: 8px; - border-bottom-left-radius: 8px; -} -.page-item:last-child .page-link { - border-top-right-radius: 8px; - border-bottom-right-radius: 8px; -} -.page-item.active .page-link { - z-index: 3; - color: #fff; - background-color: #ffcc00; - border-color: #ffcc00; -} -.page-item.disabled .page-link { - color: #6c757d; - pointer-events: none; - cursor: auto; - background-color: #fff; - border-color: #dee2e6; -} - -.pagination-lg .page-link { - padding: 0.75rem 1.5rem; - font-size: 1.125rem; - line-height: 1.5; -} -.pagination-lg .page-item:first-child .page-link { - border-top-left-radius: 8px; - border-bottom-left-radius: 8px; -} -.pagination-lg .page-item:last-child .page-link { - border-top-right-radius: 8px; - border-bottom-right-radius: 8px; -} - -.pagination-sm .page-link { - padding: 0.25rem 0.5rem; - font-size: 0.875rem; - line-height: 1.5; -} -.pagination-sm .page-item:first-child .page-link { - border-top-left-radius: 8px; - border-bottom-left-radius: 8px; -} -.pagination-sm .page-item:last-child .page-link { - border-top-right-radius: 8px; - border-bottom-right-radius: 8px; -} - -.badge { - display: inline-block; - padding: 0.25em 0.4em; - font-size: 75%; - font-weight: 700; - line-height: 1; - text-align: center; - white-space: nowrap; - vertical-align: baseline; - border-radius: 8px; - transition: color 0.15s ease-in-out, background-color 0.15s ease-in-out, border-color 0.15s ease-in-out, box-shadow 0.15s ease-in-out; -} -@media (prefers-reduced-motion: reduce) { - .badge { - transition: none; - } -} -a.badge:hover, a.badge:focus { - text-decoration: none; -} - -.badge:empty { - display: none; -} - -.btn .badge { - position: relative; - top: -1px; -} - -.badge-pill { - padding-right: 0.6em; - padding-left: 0.6em; - border-radius: 10rem; -} - -.badge-primary { - color: #495057; - background-color: #ffcc00; -} -a.badge-primary:hover, a.badge-primary:focus { - color: #495057; - background-color: #cca300; -} -a.badge-primary:focus, a.badge-primary.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.5); -} - -.badge-secondary { - color: #fff; - background-color: #212529; -} -a.badge-secondary:hover, a.badge-secondary:focus { - color: #fff; - background-color: #0a0c0d; -} -a.badge-secondary:focus, a.badge-secondary.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(33, 37, 41, 0.5); -} - -.badge-success { - color: #fff; - background-color: #28a745; -} -a.badge-success:hover, a.badge-success:focus { - color: #fff; - background-color: #1e7e34; -} -a.badge-success:focus, a.badge-success.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(40, 167, 69, 0.5); -} - -.badge-info { - color: #fff; - background-color: #17a2b8; -} -a.badge-info:hover, a.badge-info:focus { - color: #fff; - background-color: #117a8b; -} -a.badge-info:focus, a.badge-info.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(23, 162, 184, 0.5); -} - -.badge-warning { - color: #495057; - background-color: #ffc107; -} -a.badge-warning:hover, a.badge-warning:focus { - color: #495057; - background-color: #d39e00; -} -a.badge-warning:focus, a.badge-warning.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 193, 7, 0.5); -} - -.badge-danger { - color: #fff; - background-color: #dc3545; -} -a.badge-danger:hover, a.badge-danger:focus { - color: #fff; - background-color: #bd2130; -} -a.badge-danger:focus, a.badge-danger.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(220, 53, 69, 0.5); -} - -.badge-light { - color: #495057; - background-color: #f1f6f9; -} -a.badge-light:hover, a.badge-light:focus { - color: #495057; - background-color: #cddfea; -} -a.badge-light:focus, a.badge-light.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(241, 246, 249, 0.5); -} - -.badge-dark { - color: #fff; - background-color: #495057; -} -a.badge-dark:hover, a.badge-dark:focus { - color: #fff; - background-color: #32373b; -} -a.badge-dark:focus, a.badge-dark.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(73, 80, 87, 0.5); -} - -.badge-primary-light { - color: #495057; - background-color: #fffaf0; -} -a.badge-primary-light:hover, a.badge-primary-light:focus { - color: #495057; - background-color: #ffe9bd; -} -a.badge-primary-light:focus, a.badge-primary-light.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 250, 240, 0.5); -} - -.badge-secondary-light { - color: #495057; - background-color: #fff; -} -a.badge-secondary-light:hover, a.badge-secondary-light:focus { - color: #495057; - background-color: #e6e6e6; -} -a.badge-secondary-light:focus, a.badge-secondary-light.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 255, 255, 0.5); -} - -.badge-tertiary { - color: #fff; - background-color: #257af4; -} -a.badge-tertiary:hover, a.badge-tertiary:focus { - color: #fff; - background-color: #0b60db; -} -a.badge-tertiary:focus, a.badge-tertiary.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(37, 122, 244, 0.5); -} - -.badge-tertiary-light { - color: #495057; - background-color: #e3f1fe; -} -a.badge-tertiary-light:hover, a.badge-tertiary-light:focus { - color: #495057; - background-color: #b2d8fc; -} -a.badge-tertiary-light:focus, a.badge-tertiary-light.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(227, 241, 254, 0.5); -} - -.badge-white { - color: #495057; - background-color: #fff; -} -a.badge-white:hover, a.badge-white:focus { - color: #495057; - background-color: #e6e6e6; -} -a.badge-white:focus, a.badge-white.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 255, 255, 0.5); -} - -.badge-black { - color: #fff; - background-color: #212529; -} -a.badge-black:hover, a.badge-black:focus { - color: #fff; - background-color: #0a0c0d; -} -a.badge-black:focus, a.badge-black.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(33, 37, 41, 0.5); -} - -.badge-blue { - color: #fff; - background-color: #257af4; -} -a.badge-blue:hover, a.badge-blue:focus { - color: #fff; - background-color: #0b60db; -} -a.badge-blue:focus, a.badge-blue.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(37, 122, 244, 0.5); -} - -.badge-light-blue { - color: #495057; - background-color: #e3f1fe; -} -a.badge-light-blue:hover, a.badge-light-blue:focus { - color: #495057; - background-color: #b2d8fc; -} -a.badge-light-blue:focus, a.badge-light-blue.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(227, 241, 254, 0.5); -} - -.badge-yellow { - color: #495057; - background-color: #ffcc00; -} -a.badge-yellow:hover, a.badge-yellow:focus { - color: #495057; - background-color: #cca300; -} -a.badge-yellow:focus, a.badge-yellow.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 204, 0, 0.5); -} - -.badge-light-yellow { - color: #495057; - background-color: #fffaf0; -} -a.badge-light-yellow:hover, a.badge-light-yellow:focus { - color: #495057; - background-color: #ffe9bd; -} -a.badge-light-yellow:focus, a.badge-light-yellow.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 250, 240, 0.5); -} - -.badge-orange { - color: #495057; - background-color: #ff8c00; -} -a.badge-orange:hover, a.badge-orange:focus { - color: #495057; - background-color: #cc7000; -} -a.badge-orange:focus, a.badge-orange.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 140, 0, 0.5); -} - -.badge-light-orange { - color: #495057; - background-color: #ffe4b5; -} -a.badge-light-orange:hover, a.badge-light-orange:focus { - color: #495057; - background-color: #ffd182; -} -a.badge-light-orange:focus, a.badge-light-orange.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 228, 181, 0.5); -} - -.badge-red { - color: #fff; - background-color: #ff3939; -} -a.badge-red:hover, a.badge-red:focus { - color: #fff; - background-color: #ff0606; -} -a.badge-red:focus, a.badge-red.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 57, 57, 0.5); -} - -.badge-light-red { - color: #495057; - background-color: #ffe4e1; -} -a.badge-light-red:hover, a.badge-light-red:focus { - color: #495057; - background-color: #ffb6ae; -} -a.badge-light-red:focus, a.badge-light-red.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(255, 228, 225, 0.5); -} - -.badge-medium { - color: #495057; - background-color: #d6dbdf; -} -a.badge-medium:hover, a.badge-medium:focus { - color: #495057; - background-color: #b9c2c9; -} -a.badge-medium:focus, a.badge-medium.focus { - outline: 0; - box-shadow: 0 0 0 0.2rem rgba(214, 219, 223, 0.5); -} - -.jumbotron { - padding: 2rem 1rem; - margin-bottom: 2rem; - background-color: #e9ecef; - border-radius: 8px; -} -@media (min-width: 616px) { - .jumbotron { - padding: 4rem 2rem; - } -} - -.jumbotron-fluid { - padding-right: 0; - padding-left: 0; - border-radius: 0; -} - -.alert { - position: relative; - padding: 0.75rem 1.25rem; - margin-bottom: 1rem; - border: 1px solid transparent; - border-radius: 8px; -} - -.alert-heading { - color: inherit; -} - -.alert-link { - font-weight: 700; -} - -.alert-dismissible { - padding-right: 4rem; -} -.alert-dismissible .close { - position: absolute; - top: 0; - right: 0; - padding: 0.75rem 1.25rem; - color: inherit; -} - -.alert-primary { - color: #947c14; - background-color: #fff5cc; - border-color: #fff1b8; -} -.alert-primary hr { - border-top-color: #ffec9f; -} -.alert-primary .alert-link { - color: #67560e; -} - -.alert-secondary { - color: #212529; - background-color: #d3d3d4; - border-color: #c1c2c3; -} -.alert-secondary hr { - border-top-color: #b4b5b6; -} -.alert-secondary .alert-link { - color: #0a0c0d; -} - -.alert-success { - color: #256938; - background-color: #d4edda; - border-color: #c3e6cb; -} -.alert-success hr { - border-top-color: #b1dfbb; -} -.alert-success .alert-link { - color: #184324; -} - -.alert-info { - color: #1c6673; - background-color: #d1ecf1; - border-color: #bee5eb; -} -.alert-info hr { - border-top-color: #abdde5; -} -.alert-info .alert-link { - color: #12424a; -} - -.alert-warning { - color: #947617; - background-color: #fff3cd; - border-color: #ffeeba; -} -.alert-warning hr { - border-top-color: #ffe8a1; -} -.alert-warning .alert-link { - color: #685310; -} - -.alert-danger { - color: #822d38; - background-color: #f8d7da; - border-color: #f5c6cb; -} -.alert-danger hr { - border-top-color: #f1b0b7; -} -.alert-danger .alert-link { - color: #5c2028; -} - -.alert-light { - color: #8d9295; - background-color: #fcfdfe; - border-color: #fbfcfd; -} -.alert-light hr { - border-top-color: #eaeff5; -} -.alert-light .alert-link { - color: #73797c; -} - -.alert-dark { - color: #363b41; - background-color: #dbdcdd; - border-color: #ccced0; -} -.alert-dark hr { - border-top-color: #bfc1c4; -} -.alert-dark .alert-link { - color: #1f2225; -} - -.alert-primary-light { - color: #949490; - background-color: #fffefc; - border-color: #fffefb; -} -.alert-primary-light hr { - border-top-color: #fff8e2; -} -.alert-primary-light .alert-link { - color: #7b7b76; -} - -.alert-secondary-light { - color: #949698; - background-color: white; - border-color: white; -} -.alert-secondary-light hr { - border-top-color: #f2f2f2; -} -.alert-secondary-light .alert-link { - color: #7a7d7f; -} - -.alert-tertiary { - color: #235193; - background-color: #d3e4fd; - border-color: #c2dafc; -} -.alert-tertiary hr { - border-top-color: #aacbfb; -} -.alert-tertiary .alert-link { - color: #193a6a; -} - -.alert-tertiary-light { - color: #868f98; - background-color: #f9fcff; - border-color: #f7fbff; -} -.alert-tertiary-light hr { - border-top-color: #deeeff; -} -.alert-tertiary-light .alert-link { - color: #6c767f; -} - -.alert-white { - color: #949698; - background-color: white; - border-color: white; -} -.alert-white hr { - border-top-color: #f2f2f2; -} -.alert-white .alert-link { - color: #7a7d7f; -} - -.alert-black { - color: #212529; - background-color: #d3d3d4; - border-color: #c1c2c3; -} -.alert-black hr { - border-top-color: #b4b5b6; -} -.alert-black .alert-link { - color: #0a0c0d; -} - -.alert-blue { - color: #235193; - background-color: #d3e4fd; - border-color: #c2dafc; -} -.alert-blue hr { - border-top-color: #aacbfb; -} -.alert-blue .alert-link { - color: #193a6a; -} - -.alert-light-blue { - color: #868f98; - background-color: #f9fcff; - border-color: #f7fbff; -} -.alert-light-blue hr { - border-top-color: #deeeff; -} -.alert-light-blue .alert-link { - color: #6c767f; -} - -.alert-yellow { - color: #947c14; - background-color: #fff5cc; - border-color: #fff1b8; -} -.alert-yellow hr { - border-top-color: #ffec9f; -} -.alert-yellow .alert-link { - color: #67560e; -} - -.alert-light-yellow { - color: #949490; - background-color: #fffefc; - border-color: #fffefb; -} -.alert-light-yellow hr { - border-top-color: #fff8e2; -} -.alert-light-yellow .alert-link { - color: #7b7b76; -} - -.alert-orange { - color: #945b14; - background-color: #ffe8cc; - border-color: #ffdfb8; -} -.alert-orange hr { - border-top-color: #ffd49f; -} -.alert-orange .alert-link { - color: #673f0e; -} - -.alert-light-orange { - color: #948872; - background-color: floralwhite; - border-color: #fff7ea; -} -.alert-light-orange hr { - border-top-color: #ffedd1; -} -.alert-light-orange .alert-link { - color: #786e5b; -} - -.alert-red { - color: #942f31; - background-color: #ffd7d7; - border-color: #ffc8c8; -} -.alert-red hr { - border-top-color: #ffafaf; -} -.alert-red .alert-link { - color: #6d2324; -} - -.alert-light-red { - color: #948889; - background-color: #fffaf9; - border-color: #fff7f7; -} -.alert-light-red hr { - border-top-color: #ffdede; -} -.alert-light-red .alert-link { - color: #7b6e6f; -} - -.alert-medium { - color: #7f8488; - background-color: #f7f8f9; - border-color: #f4f5f6; -} -.alert-medium hr { - border-top-color: #e6e8eb; -} -.alert-medium .alert-link { - color: #666a6e; -} - -@-webkit-keyframes progress-bar-stripes { - from { - background-position: 1rem 0; - } - to { - background-position: 0 0; - } -} - -@keyframes progress-bar-stripes { - from { - background-position: 1rem 0; - } - to { - background-position: 0 0; - } -} -.progress { - display: flex; - height: 1rem; - overflow: hidden; - font-size: 0.75rem; - background-color: #e9ecef; - border-radius: 8px; -} - -.progress-bar { - display: flex; - flex-direction: column; - justify-content: center; - overflow: hidden; - color: #fff; - text-align: center; - white-space: nowrap; - background-color: #ffcc00; - transition: width 0.6s ease; -} -@media (prefers-reduced-motion: reduce) { - .progress-bar { - transition: none; - } -} - -.progress-bar-striped { - background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); - background-size: 1rem 1rem; -} - -.progress-bar-animated { - -webkit-animation: progress-bar-stripes 1s linear infinite; - animation: progress-bar-stripes 1s linear infinite; -} -@media (prefers-reduced-motion: reduce) { - .progress-bar-animated { - -webkit-animation: none; - animation: none; - } -} - -.media { - display: flex; - align-items: flex-start; -} - -.media-body { - flex: 1; -} - -.list-group { - display: flex; - flex-direction: column; - padding-left: 0; - margin-bottom: 0; -} - -.list-group-item-action { - width: 100%; - color: #6c757d; - text-align: inherit; -} -.list-group-item-action:hover, .list-group-item-action:focus { - z-index: 1; - color: #6c757d; - text-decoration: none; - background-color: #f1f6f9; -} -.list-group-item-action:active { - color: #212529; - background-color: #e9ecef; -} - -.list-group-item { - position: relative; - display: block; - padding: 0.75rem 1.25rem; - background-color: #fff; - border: 1px solid rgba(33, 37, 41, 0.125); -} -.list-group-item:first-child { - border-top-left-radius: 8px; - border-top-right-radius: 8px; -} -.list-group-item:last-child { - border-bottom-right-radius: 8px; - border-bottom-left-radius: 8px; -} -.list-group-item.disabled, .list-group-item:disabled { - color: #6c757d; - pointer-events: none; - background-color: #fff; -} -.list-group-item.active { - z-index: 2; - color: #fff; - background-color: #ffcc00; - border-color: #ffcc00; -} -.list-group-item + .list-group-item { - border-top-width: 0; -} -.list-group-item + .list-group-item.active { - margin-top: -1px; - border-top-width: 1px; -} - -.list-group-horizontal { - flex-direction: row; -} -.list-group-horizontal .list-group-item:first-child { - border-bottom-left-radius: 8px; - border-top-right-radius: 0; -} -.list-group-horizontal .list-group-item:last-child { - border-top-right-radius: 8px; - border-bottom-left-radius: 0; -} -.list-group-horizontal .list-group-item.active { - margin-top: 0; -} -.list-group-horizontal .list-group-item + .list-group-item { - border-top-width: 1px; - border-left-width: 0; -} -.list-group-horizontal .list-group-item + .list-group-item.active { - margin-left: -1px; - border-left-width: 1px; -} - -@media (min-width: 400px) { - .list-group-horizontal-xs { - flex-direction: row; - } - .list-group-horizontal-xs .list-group-item:first-child { - border-bottom-left-radius: 8px; - border-top-right-radius: 0; - } - .list-group-horizontal-xs .list-group-item:last-child { - border-top-right-radius: 8px; - border-bottom-left-radius: 0; - } - .list-group-horizontal-xs .list-group-item.active { - margin-top: 0; - } - .list-group-horizontal-xs .list-group-item + .list-group-item { - border-top-width: 1px; - border-left-width: 0; - } - .list-group-horizontal-xs .list-group-item + .list-group-item.active { - margin-left: -1px; - border-left-width: 1px; - } -} -@media (min-width: 616px) { - .list-group-horizontal-sm { - flex-direction: row; - } - .list-group-horizontal-sm .list-group-item:first-child { - border-bottom-left-radius: 8px; - border-top-right-radius: 0; - } - .list-group-horizontal-sm .list-group-item:last-child { - border-top-right-radius: 8px; - border-bottom-left-radius: 0; - } - .list-group-horizontal-sm .list-group-item.active { - margin-top: 0; - } - .list-group-horizontal-sm .list-group-item + .list-group-item { - border-top-width: 1px; - border-left-width: 0; - } - .list-group-horizontal-sm .list-group-item + .list-group-item.active { - margin-left: -1px; - border-left-width: 1px; - } -} -@media (min-width: 768px) { - .list-group-horizontal-md { - flex-direction: row; - } - .list-group-horizontal-md .list-group-item:first-child { - border-bottom-left-radius: 8px; - border-top-right-radius: 0; - } - .list-group-horizontal-md .list-group-item:last-child { - border-top-right-radius: 8px; - border-bottom-left-radius: 0; - } - .list-group-horizontal-md .list-group-item.active { - margin-top: 0; - } - .list-group-horizontal-md .list-group-item + .list-group-item { - border-top-width: 1px; - border-left-width: 0; - } - .list-group-horizontal-md .list-group-item + .list-group-item.active { - margin-left: -1px; - border-left-width: 1px; - } -} -@media (min-width: 980px) { - .list-group-horizontal-lg { - flex-direction: row; - } - .list-group-horizontal-lg .list-group-item:first-child { - border-bottom-left-radius: 8px; - border-top-right-radius: 0; - } - .list-group-horizontal-lg .list-group-item:last-child { - border-top-right-radius: 8px; - border-bottom-left-radius: 0; - } - .list-group-horizontal-lg .list-group-item.active { - margin-top: 0; - } - .list-group-horizontal-lg .list-group-item + .list-group-item { - border-top-width: 1px; - border-left-width: 0; - } - .list-group-horizontal-lg .list-group-item + .list-group-item.active { - margin-left: -1px; - border-left-width: 1px; - } -} -@media (min-width: 1240px) { - .list-group-horizontal-xl { - flex-direction: row; - } - .list-group-horizontal-xl .list-group-item:first-child { - border-bottom-left-radius: 8px; - border-top-right-radius: 0; - } - .list-group-horizontal-xl .list-group-item:last-child { - border-top-right-radius: 8px; - border-bottom-left-radius: 0; - } - .list-group-horizontal-xl .list-group-item.active { - margin-top: 0; - } - .list-group-horizontal-xl .list-group-item + .list-group-item { - border-top-width: 1px; - border-left-width: 0; - } - .list-group-horizontal-xl .list-group-item + .list-group-item.active { - margin-left: -1px; - border-left-width: 1px; - } -} -.list-group-flush .list-group-item { - border-right-width: 0; - border-left-width: 0; - border-radius: 0; -} -.list-group-flush .list-group-item:first-child { - border-top-width: 0; -} -.list-group-flush:last-child .list-group-item:last-child { - border-bottom-width: 0; -} - -.list-group-item-primary { - color: #947c14; - background-color: #fff1b8; -} -.list-group-item-primary.list-group-item-action:hover, .list-group-item-primary.list-group-item-action:focus { - color: #947c14; - background-color: #ffec9f; -} -.list-group-item-primary.list-group-item-action.active { - color: #fff; - background-color: #947c14; - border-color: #947c14; -} - -.list-group-item-secondary { - color: #212529; - background-color: #c1c2c3; -} -.list-group-item-secondary.list-group-item-action:hover, .list-group-item-secondary.list-group-item-action:focus { - color: #212529; - background-color: #b4b5b6; -} -.list-group-item-secondary.list-group-item-action.active { - color: #fff; - background-color: #212529; - border-color: #212529; -} - -.list-group-item-success { - color: #256938; - background-color: #c3e6cb; -} -.list-group-item-success.list-group-item-action:hover, .list-group-item-success.list-group-item-action:focus { - color: #256938; - background-color: #b1dfbb; -} -.list-group-item-success.list-group-item-action.active { - color: #fff; - background-color: #256938; - border-color: #256938; -} - -.list-group-item-info { - color: #1c6673; - background-color: #bee5eb; -} -.list-group-item-info.list-group-item-action:hover, .list-group-item-info.list-group-item-action:focus { - color: #1c6673; - background-color: #abdde5; -} -.list-group-item-info.list-group-item-action.active { - color: #fff; - background-color: #1c6673; - border-color: #1c6673; -} - -.list-group-item-warning { - color: #947617; - background-color: #ffeeba; -} -.list-group-item-warning.list-group-item-action:hover, .list-group-item-warning.list-group-item-action:focus { - color: #947617; - background-color: #ffe8a1; -} -.list-group-item-warning.list-group-item-action.active { - color: #fff; - background-color: #947617; - border-color: #947617; -} - -.list-group-item-danger { - color: #822d38; - background-color: #f5c6cb; -} -.list-group-item-danger.list-group-item-action:hover, .list-group-item-danger.list-group-item-action:focus { - color: #822d38; - background-color: #f1b0b7; -} -.list-group-item-danger.list-group-item-action.active { - color: #fff; - background-color: #822d38; - border-color: #822d38; -} - -.list-group-item-light { - color: #8d9295; - background-color: #fbfcfd; -} -.list-group-item-light.list-group-item-action:hover, .list-group-item-light.list-group-item-action:focus { - color: #8d9295; - background-color: #eaeff5; -} -.list-group-item-light.list-group-item-action.active { - color: #fff; - background-color: #8d9295; - border-color: #8d9295; -} - -.list-group-item-dark { - color: #363b41; - background-color: #ccced0; -} -.list-group-item-dark.list-group-item-action:hover, .list-group-item-dark.list-group-item-action:focus { - color: #363b41; - background-color: #bfc1c4; -} -.list-group-item-dark.list-group-item-action.active { - color: #fff; - background-color: #363b41; - border-color: #363b41; -} - -.list-group-item-primary-light { - color: #949490; - background-color: #fffefb; -} -.list-group-item-primary-light.list-group-item-action:hover, .list-group-item-primary-light.list-group-item-action:focus { - color: #949490; - background-color: #fff8e2; -} -.list-group-item-primary-light.list-group-item-action.active { - color: #fff; - background-color: #949490; - border-color: #949490; -} - -.list-group-item-secondary-light { - color: #949698; - background-color: white; -} -.list-group-item-secondary-light.list-group-item-action:hover, .list-group-item-secondary-light.list-group-item-action:focus { - color: #949698; - background-color: #f2f2f2; -} -.list-group-item-secondary-light.list-group-item-action.active { - color: #fff; - background-color: #949698; - border-color: #949698; -} - -.list-group-item-tertiary { - color: #235193; - background-color: #c2dafc; -} -.list-group-item-tertiary.list-group-item-action:hover, .list-group-item-tertiary.list-group-item-action:focus { - color: #235193; - background-color: #aacbfb; -} -.list-group-item-tertiary.list-group-item-action.active { - color: #fff; - background-color: #235193; - border-color: #235193; -} - -.list-group-item-tertiary-light { - color: #868f98; - background-color: #f7fbff; -} -.list-group-item-tertiary-light.list-group-item-action:hover, .list-group-item-tertiary-light.list-group-item-action:focus { - color: #868f98; - background-color: #deeeff; -} -.list-group-item-tertiary-light.list-group-item-action.active { - color: #fff; - background-color: #868f98; - border-color: #868f98; -} - -.list-group-item-white { - color: #949698; - background-color: white; -} -.list-group-item-white.list-group-item-action:hover, .list-group-item-white.list-group-item-action:focus { - color: #949698; - background-color: #f2f2f2; -} -.list-group-item-white.list-group-item-action.active { - color: #fff; - background-color: #949698; - border-color: #949698; -} - -.list-group-item-black { - color: #212529; - background-color: #c1c2c3; -} -.list-group-item-black.list-group-item-action:hover, .list-group-item-black.list-group-item-action:focus { - color: #212529; - background-color: #b4b5b6; -} -.list-group-item-black.list-group-item-action.active { - color: #fff; - background-color: #212529; - border-color: #212529; -} - -.list-group-item-blue { - color: #235193; - background-color: #c2dafc; -} -.list-group-item-blue.list-group-item-action:hover, .list-group-item-blue.list-group-item-action:focus { - color: #235193; - background-color: #aacbfb; -} -.list-group-item-blue.list-group-item-action.active { - color: #fff; - background-color: #235193; - border-color: #235193; -} - -.list-group-item-light-blue { - color: #868f98; - background-color: #f7fbff; -} -.list-group-item-light-blue.list-group-item-action:hover, .list-group-item-light-blue.list-group-item-action:focus { - color: #868f98; - background-color: #deeeff; -} -.list-group-item-light-blue.list-group-item-action.active { - color: #fff; - background-color: #868f98; - border-color: #868f98; -} - -.list-group-item-yellow { - color: #947c14; - background-color: #fff1b8; -} -.list-group-item-yellow.list-group-item-action:hover, .list-group-item-yellow.list-group-item-action:focus { - color: #947c14; - background-color: #ffec9f; -} -.list-group-item-yellow.list-group-item-action.active { - color: #fff; - background-color: #947c14; - border-color: #947c14; -} - -.list-group-item-light-yellow { - color: #949490; - background-color: #fffefb; -} -.list-group-item-light-yellow.list-group-item-action:hover, .list-group-item-light-yellow.list-group-item-action:focus { - color: #949490; - background-color: #fff8e2; -} -.list-group-item-light-yellow.list-group-item-action.active { - color: #fff; - background-color: #949490; - border-color: #949490; -} - -.list-group-item-orange { - color: #945b14; - background-color: #ffdfb8; -} -.list-group-item-orange.list-group-item-action:hover, .list-group-item-orange.list-group-item-action:focus { - color: #945b14; - background-color: #ffd49f; -} -.list-group-item-orange.list-group-item-action.active { - color: #fff; - background-color: #945b14; - border-color: #945b14; -} - -.list-group-item-light-orange { - color: #948872; - background-color: #fff7ea; -} -.list-group-item-light-orange.list-group-item-action:hover, .list-group-item-light-orange.list-group-item-action:focus { - color: #948872; - background-color: #ffedd1; -} -.list-group-item-light-orange.list-group-item-action.active { - color: #fff; - background-color: #948872; - border-color: #948872; -} - -.list-group-item-red { - color: #942f31; - background-color: #ffc8c8; -} -.list-group-item-red.list-group-item-action:hover, .list-group-item-red.list-group-item-action:focus { - color: #942f31; - background-color: #ffafaf; -} -.list-group-item-red.list-group-item-action.active { - color: #fff; - background-color: #942f31; - border-color: #942f31; -} - -.list-group-item-light-red { - color: #948889; - background-color: #fff7f7; -} -.list-group-item-light-red.list-group-item-action:hover, .list-group-item-light-red.list-group-item-action:focus { - color: #948889; - background-color: #ffdede; -} -.list-group-item-light-red.list-group-item-action.active { - color: #fff; - background-color: #948889; - border-color: #948889; -} - -.list-group-item-medium { - color: #7f8488; - background-color: #f4f5f6; -} -.list-group-item-medium.list-group-item-action:hover, .list-group-item-medium.list-group-item-action:focus { - color: #7f8488; - background-color: #e6e8eb; -} -.list-group-item-medium.list-group-item-action.active { - color: #fff; - background-color: #7f8488; - border-color: #7f8488; -} - -.close { - float: right; - font-size: 1.5rem; - font-weight: 700; - line-height: 1; - color: #212529; - text-shadow: 0 1px 0 #fff; - opacity: 0.5; -} -@media (max-width: 1200px) { - .close { - font-size: calc(1.275rem + 0.3vw); - } -} -.close:hover { - color: #212529; - text-decoration: none; -} -.close:not(:disabled):not(.disabled):hover, .close:not(:disabled):not(.disabled):focus { - opacity: 0.75; -} - -button.close { - padding: 0; - background-color: transparent; - border: 0; - -webkit-appearance: none; - -moz-appearance: none; - appearance: none; -} - -a.close.disabled { - pointer-events: none; -} - -.toast { - max-width: 350px; - overflow: hidden; - font-size: 0.875rem; - background-color: rgba(255, 255, 255, 0.85); - background-clip: padding-box; - border: 1px solid rgba(0, 0, 0, 0.1); - box-shadow: 0 0.25rem 0.75rem rgba(33, 37, 41, 0.1); - -webkit-backdrop-filter: blur(10px); - backdrop-filter: blur(10px); - opacity: 0; - border-radius: 0.25rem; -} -.toast:not(:last-child) { - margin-bottom: 0.75rem; -} -.toast.showing { - opacity: 1; -} -.toast.show { - display: block; - opacity: 1; -} -.toast.hide { - display: none; -} - -.toast-header { - display: flex; - align-items: center; - padding: 0.25rem 0.75rem; - color: #6c757d; - background-color: rgba(255, 255, 255, 0.85); - background-clip: padding-box; - border-bottom: 1px solid rgba(0, 0, 0, 0.05); -} - -.toast-body { - padding: 0.75rem; -} - -.modal-open { - overflow: hidden; -} -.modal-open .modal { - overflow-x: hidden; - overflow-y: auto; -} - -.modal { - position: fixed; - top: 0; - left: 0; - z-index: 1050; - display: none; - width: 100%; - height: 100%; - overflow: hidden; - outline: 0; -} - -.modal-dialog { - position: relative; - width: auto; - margin: 0.5rem; - pointer-events: none; -} -.modal.fade .modal-dialog { - transition: transform 0.3s ease-out; - transform: translate(0, -50px); -} -@media (prefers-reduced-motion: reduce) { - .modal.fade .modal-dialog { - transition: none; - } -} -.modal.show .modal-dialog { - transform: none; -} -.modal.modal-static .modal-dialog { - transform: scale(1.02); -} - -.modal-dialog-scrollable { - display: flex; - max-height: calc(100% - 1rem); -} -.modal-dialog-scrollable .modal-content { - max-height: calc(100vh - 1rem); - overflow: hidden; -} -.modal-dialog-scrollable .modal-header, -.modal-dialog-scrollable .modal-footer { - flex-shrink: 0; -} -.modal-dialog-scrollable .modal-body { - overflow-y: auto; -} - -.modal-dialog-centered { - display: flex; - align-items: center; - min-height: calc(100% - 1rem); -} -.modal-dialog-centered::before { - display: block; - height: calc(100vh - 1rem); - content: ""; -} -.modal-dialog-centered.modal-dialog-scrollable { - flex-direction: column; - justify-content: center; - height: 100%; -} -.modal-dialog-centered.modal-dialog-scrollable .modal-content { - max-height: none; -} -.modal-dialog-centered.modal-dialog-scrollable::before { - content: none; -} - -.modal-content { - position: relative; - display: flex; - flex-direction: column; - width: 100%; - pointer-events: auto; - background-color: #fff; - background-clip: padding-box; - border: 1px solid rgba(33, 37, 41, 0.2); - border-radius: 8px; - outline: 0; -} - -.modal-backdrop { - position: fixed; - top: 0; - left: 0; - z-index: 1040; - width: 100vw; - height: 100vh; - background-color: #212529; -} -.modal-backdrop.fade { - opacity: 0; -} -.modal-backdrop.show { - opacity: 0.5; -} - -.modal-header { - display: flex; - align-items: flex-start; - justify-content: space-between; - padding: 1rem 1rem; - border-bottom: 1px solid #d6dbdf; - border-top-left-radius: 7px; - border-top-right-radius: 7px; -} -.modal-header .close { - padding: 1rem 1rem; - margin: -1rem -1rem -1rem auto; -} - -.modal-title { - margin-bottom: 0; - line-height: 1.5; -} - -.modal-body { - position: relative; - flex: 1 1 auto; - padding: 1rem; -} - -.modal-footer { - display: flex; - flex-wrap: wrap; - align-items: center; - justify-content: flex-end; - padding: 0.75rem; - border-top: 1px solid #d6dbdf; - border-bottom-right-radius: 7px; - border-bottom-left-radius: 7px; -} -.modal-footer > * { - margin: 0.25rem; -} - -.modal-scrollbar-measure { - position: absolute; - top: -9999px; - width: 50px; - height: 50px; - overflow: scroll; -} - -@media (min-width: 616px) { - .modal-dialog { - max-width: 500px; - margin: 1.75rem auto; - } - - .modal-dialog-scrollable { - max-height: calc(100% - 3.5rem); - } - .modal-dialog-scrollable .modal-content { - max-height: calc(100vh - 3.5rem); - } - - .modal-dialog-centered { - min-height: calc(100% - 3.5rem); - } - .modal-dialog-centered::before { - height: calc(100vh - 3.5rem); - } - - .modal-sm { - max-width: 300px; - } -} -@media (min-width: 980px) { - .modal-lg, -.modal-xl { - max-width: 800px; - } -} -@media (min-width: 1240px) { - .modal-xl { - max-width: 1140px; - } -} -.tooltip { - position: absolute; - z-index: 1070; - display: block; - margin: 0; - font-family: "Noto Sans", sans-serif; - font-style: normal; - font-weight: 400; - line-height: 1.5; - text-align: left; - text-align: start; - text-decoration: none; - text-shadow: none; - text-transform: none; - letter-spacing: normal; - word-break: normal; - word-spacing: normal; - white-space: normal; - line-break: auto; - font-size: 0.875rem; - word-wrap: break-word; - opacity: 0; -} -.tooltip.show { - opacity: 0.9; -} -.tooltip .arrow { - position: absolute; - display: block; - width: 0.8rem; - height: 0.4rem; -} -.tooltip .arrow::before { - position: absolute; - content: ""; - border-color: transparent; - border-style: solid; -} - -.bs-tooltip-top, .bs-tooltip-auto[x-placement^=top] { - padding: 0.4rem 0; -} -.bs-tooltip-top .arrow, .bs-tooltip-auto[x-placement^=top] .arrow { - bottom: 0; -} -.bs-tooltip-top .arrow::before, .bs-tooltip-auto[x-placement^=top] .arrow::before { - top: 0; - border-width: 0.4rem 0.4rem 0; - border-top-color: #212529; -} - -.bs-tooltip-right, .bs-tooltip-auto[x-placement^=right] { - padding: 0 0.4rem; -} -.bs-tooltip-right .arrow, .bs-tooltip-auto[x-placement^=right] .arrow { - left: 0; - width: 0.4rem; - height: 0.8rem; -} -.bs-tooltip-right .arrow::before, .bs-tooltip-auto[x-placement^=right] .arrow::before { - right: 0; - border-width: 0.4rem 0.4rem 0.4rem 0; - border-right-color: #212529; -} - -.bs-tooltip-bottom, .bs-tooltip-auto[x-placement^=bottom] { - padding: 0.4rem 0; -} -.bs-tooltip-bottom .arrow, .bs-tooltip-auto[x-placement^=bottom] .arrow { - top: 0; -} -.bs-tooltip-bottom .arrow::before, .bs-tooltip-auto[x-placement^=bottom] .arrow::before { - bottom: 0; - border-width: 0 0.4rem 0.4rem; - border-bottom-color: #212529; -} - -.bs-tooltip-left, .bs-tooltip-auto[x-placement^=left] { - padding: 0 0.4rem; -} -.bs-tooltip-left .arrow, .bs-tooltip-auto[x-placement^=left] .arrow { - right: 0; - width: 0.4rem; - height: 0.8rem; -} -.bs-tooltip-left .arrow::before, .bs-tooltip-auto[x-placement^=left] .arrow::before { - left: 0; - border-width: 0.4rem 0 0.4rem 0.4rem; - border-left-color: #212529; -} - -.tooltip-inner { - max-width: 200px; - padding: 0.25rem 0.5rem; - color: #fff; - text-align: center; - background-color: #212529; - border-radius: 8px; -} - -.popover { - position: absolute; - top: 0; - left: 0; - z-index: 1060; - display: block; - max-width: 276px; - font-family: "Noto Sans", sans-serif; - font-style: normal; - font-weight: 400; - line-height: 1.5; - text-align: left; - text-align: start; - text-decoration: none; - text-shadow: none; - text-transform: none; - letter-spacing: normal; - word-break: normal; - word-spacing: normal; - white-space: normal; - line-break: auto; - font-size: 0.875rem; - word-wrap: break-word; - background-color: #fff; - background-clip: padding-box; - border: 1px solid rgba(33, 37, 41, 0.2); - border-radius: 8px; -} -.popover .arrow { - position: absolute; - display: block; - width: 1rem; - height: 0.5rem; - margin: 0 8px; -} -.popover .arrow::before, .popover .arrow::after { - position: absolute; - display: block; - content: ""; - border-color: transparent; - border-style: solid; -} - -.bs-popover-top, .bs-popover-auto[x-placement^=top] { - margin-bottom: 0.5rem; -} -.bs-popover-top > .arrow, .bs-popover-auto[x-placement^=top] > .arrow { - bottom: calc(-0.5rem - 1px); -} -.bs-popover-top > .arrow::before, .bs-popover-auto[x-placement^=top] > .arrow::before { - bottom: 0; - border-width: 0.5rem 0.5rem 0; - border-top-color: rgba(33, 37, 41, 0.25); -} -.bs-popover-top > .arrow::after, .bs-popover-auto[x-placement^=top] > .arrow::after { - bottom: 1px; - border-width: 0.5rem 0.5rem 0; - border-top-color: #fff; -} - -.bs-popover-right, .bs-popover-auto[x-placement^=right] { - margin-left: 0.5rem; -} -.bs-popover-right > .arrow, .bs-popover-auto[x-placement^=right] > .arrow { - left: calc(-0.5rem - 1px); - width: 0.5rem; - height: 1rem; - margin: 8px 0; -} -.bs-popover-right > .arrow::before, .bs-popover-auto[x-placement^=right] > .arrow::before { - left: 0; - border-width: 0.5rem 0.5rem 0.5rem 0; - border-right-color: rgba(33, 37, 41, 0.25); -} -.bs-popover-right > .arrow::after, .bs-popover-auto[x-placement^=right] > .arrow::after { - left: 1px; - border-width: 0.5rem 0.5rem 0.5rem 0; - border-right-color: #fff; -} - -.bs-popover-bottom, .bs-popover-auto[x-placement^=bottom] { - margin-top: 0.5rem; -} -.bs-popover-bottom > .arrow, .bs-popover-auto[x-placement^=bottom] > .arrow { - top: calc(-0.5rem - 1px); -} -.bs-popover-bottom > .arrow::before, .bs-popover-auto[x-placement^=bottom] > .arrow::before { - top: 0; - border-width: 0 0.5rem 0.5rem 0.5rem; - border-bottom-color: rgba(33, 37, 41, 0.25); -} -.bs-popover-bottom > .arrow::after, .bs-popover-auto[x-placement^=bottom] > .arrow::after { - top: 1px; - border-width: 0 0.5rem 0.5rem 0.5rem; - border-bottom-color: #fff; -} -.bs-popover-bottom .popover-header::before, .bs-popover-auto[x-placement^=bottom] .popover-header::before { - position: absolute; - top: 0; - left: 50%; - display: block; - width: 1rem; - margin-left: -0.5rem; - content: ""; - border-bottom: 1px solid #f7f7f7; -} - -.bs-popover-left, .bs-popover-auto[x-placement^=left] { - margin-right: 0.5rem; -} -.bs-popover-left > .arrow, .bs-popover-auto[x-placement^=left] > .arrow { - right: calc(-0.5rem - 1px); - width: 0.5rem; - height: 1rem; - margin: 8px 0; -} -.bs-popover-left > .arrow::before, .bs-popover-auto[x-placement^=left] > .arrow::before { - right: 0; - border-width: 0.5rem 0 0.5rem 0.5rem; - border-left-color: rgba(33, 37, 41, 0.25); -} -.bs-popover-left > .arrow::after, .bs-popover-auto[x-placement^=left] > .arrow::after { - right: 1px; - border-width: 0.5rem 0 0.5rem 0.5rem; - border-left-color: #fff; -} - -.popover-header { - padding: 0.5rem 0.75rem; - margin-bottom: 0; - font-size: 1rem; - background-color: #f7f7f7; - border-bottom: 1px solid #ebebeb; - border-top-left-radius: 7px; - border-top-right-radius: 7px; -} -.popover-header:empty { - display: none; -} - -.popover-body { - padding: 0.5rem 0.75rem; - color: #212529; -} - -.carousel { - position: relative; -} - -.carousel.pointer-event { - touch-action: pan-y; -} - -.carousel-inner { - position: relative; - width: 100%; - overflow: hidden; -} -.carousel-inner::after { - display: block; - clear: both; - content: ""; -} - -.carousel-item { - position: relative; - display: none; - float: left; - width: 100%; - margin-right: -100%; - -webkit-backface-visibility: hidden; - backface-visibility: hidden; - transition: transform 0.6s ease-in-out; -} -@media (prefers-reduced-motion: reduce) { - .carousel-item { - transition: none; - } -} - -.carousel-item.active, -.carousel-item-next, -.carousel-item-prev { - display: block; -} - -.carousel-item-next:not(.carousel-item-left), -.active.carousel-item-right { - transform: translateX(100%); -} - -.carousel-item-prev:not(.carousel-item-right), -.active.carousel-item-left { - transform: translateX(-100%); -} - -.carousel-fade .carousel-item { - opacity: 0; - transition-property: opacity; - transform: none; -} -.carousel-fade .carousel-item.active, -.carousel-fade .carousel-item-next.carousel-item-left, -.carousel-fade .carousel-item-prev.carousel-item-right { - z-index: 1; - opacity: 1; -} -.carousel-fade .active.carousel-item-left, -.carousel-fade .active.carousel-item-right { - z-index: 0; - opacity: 0; - transition: opacity 0s 0.6s; -} -@media (prefers-reduced-motion: reduce) { - .carousel-fade .active.carousel-item-left, -.carousel-fade .active.carousel-item-right { - transition: none; - } -} - -.carousel-control-prev, -.carousel-control-next { - position: absolute; - top: 0; - bottom: 0; - z-index: 1; - display: flex; - align-items: center; - justify-content: center; - width: 15%; - color: #fff; - text-align: center; - opacity: 0.5; - transition: opacity 0.15s ease; -} -@media (prefers-reduced-motion: reduce) { - .carousel-control-prev, -.carousel-control-next { - transition: none; - } -} -.carousel-control-prev:hover, .carousel-control-prev:focus, -.carousel-control-next:hover, -.carousel-control-next:focus { - color: #fff; - text-decoration: none; - outline: 0; - opacity: 0.9; -} - -.carousel-control-prev { - left: 0; -} - -.carousel-control-next { - right: 0; -} - -.carousel-control-prev-icon, -.carousel-control-next-icon { - display: inline-block; - width: 20px; - height: 20px; - background: no-repeat 50%/100% 100%; -} - -.carousel-control-prev-icon { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath d='M5.25 0l-4 4 4 4 1.5-1.5L4.25 4l2.5-2.5L5.25 0z'/%3e%3c/svg%3e"); -} - -.carousel-control-next-icon { - background-image: url("data:image/svg+xml,%3csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8' viewBox='0 0 8 8'%3e%3cpath d='M2.75 0l-1.5 1.5L3.75 4l-2.5 2.5L2.75 8l4-4-4-4z'/%3e%3c/svg%3e"); -} - -.carousel-indicators { - position: absolute; - right: 0; - bottom: 0; - left: 0; - z-index: 15; - display: flex; - justify-content: center; - padding-left: 0; - margin-right: 15%; - margin-left: 15%; - list-style: none; -} -.carousel-indicators li { - box-sizing: content-box; - flex: 0 1 auto; - width: 30px; - height: 3px; - margin-right: 3px; - margin-left: 3px; - text-indent: -999px; - cursor: pointer; - background-color: #fff; - background-clip: padding-box; - border-top: 10px solid transparent; - border-bottom: 10px solid transparent; - opacity: 0.5; - transition: opacity 0.6s ease; -} -@media (prefers-reduced-motion: reduce) { - .carousel-indicators li { - transition: none; - } -} -.carousel-indicators .active { - opacity: 1; -} - -.carousel-caption { - position: absolute; - right: 15%; - bottom: 20px; - left: 15%; - z-index: 10; - padding-top: 20px; - padding-bottom: 20px; - color: #fff; - text-align: center; -} - -@-webkit-keyframes spinner-border { - to { - transform: rotate(360deg); - } -} - -@keyframes spinner-border { - to { - transform: rotate(360deg); - } -} -.spinner-border { - display: inline-block; - width: 2rem; - height: 2rem; - vertical-align: text-bottom; - border: 0.25em solid currentColor; - border-right-color: transparent; - border-radius: 50%; - -webkit-animation: spinner-border 0.75s linear infinite; - animation: spinner-border 0.75s linear infinite; -} - -.spinner-border-sm { - width: 1rem; - height: 1rem; - border-width: 0.2em; -} - -@-webkit-keyframes spinner-grow { - 0% { - transform: scale(0); - } - 50% { - opacity: 1; - } -} - -@keyframes spinner-grow { - 0% { - transform: scale(0); - } - 50% { - opacity: 1; - } -} -.spinner-grow { - display: inline-block; - width: 2rem; - height: 2rem; - vertical-align: text-bottom; - background-color: currentColor; - border-radius: 50%; - opacity: 0; - -webkit-animation: spinner-grow 0.75s linear infinite; - animation: spinner-grow 0.75s linear infinite; -} - -.spinner-grow-sm { - width: 1rem; - height: 1rem; -} - -.align-baseline { - vertical-align: baseline !important; -} - -.align-top { - vertical-align: top !important; -} - -.align-middle { - vertical-align: middle !important; -} - -.align-bottom { - vertical-align: bottom !important; -} - -.align-text-bottom { - vertical-align: text-bottom !important; -} - -.align-text-top { - vertical-align: text-top !important; -} - -.bg-primary { - background-color: #ffcc00 !important; -} - -a.bg-primary:hover, a.bg-primary:focus, -button.bg-primary:hover, -button.bg-primary:focus { - background-color: #cca300 !important; -} - -.bg-secondary { - background-color: #212529 !important; -} - -a.bg-secondary:hover, a.bg-secondary:focus, -button.bg-secondary:hover, -button.bg-secondary:focus { - background-color: #0a0c0d !important; -} - -.bg-success { - background-color: #28a745 !important; -} - -a.bg-success:hover, a.bg-success:focus, -button.bg-success:hover, -button.bg-success:focus { - background-color: #1e7e34 !important; -} - -.bg-info { - background-color: #17a2b8 !important; -} - -a.bg-info:hover, a.bg-info:focus, -button.bg-info:hover, -button.bg-info:focus { - background-color: #117a8b !important; -} - -.bg-warning { - background-color: #ffc107 !important; -} - -a.bg-warning:hover, a.bg-warning:focus, -button.bg-warning:hover, -button.bg-warning:focus { - background-color: #d39e00 !important; -} - -.bg-danger { - background-color: #dc3545 !important; -} - -a.bg-danger:hover, a.bg-danger:focus, -button.bg-danger:hover, -button.bg-danger:focus { - background-color: #bd2130 !important; -} - -.bg-light { - background-color: #f1f6f9 !important; -} - -a.bg-light:hover, a.bg-light:focus, -button.bg-light:hover, -button.bg-light:focus { - background-color: #cddfea !important; -} - -.bg-dark { - background-color: #495057 !important; -} - -a.bg-dark:hover, a.bg-dark:focus, -button.bg-dark:hover, -button.bg-dark:focus { - background-color: #32373b !important; -} - -.bg-primary-light { - background-color: #fffaf0 !important; -} - -a.bg-primary-light:hover, a.bg-primary-light:focus, -button.bg-primary-light:hover, -button.bg-primary-light:focus { - background-color: #ffe9bd !important; -} - -.bg-secondary-light { - background-color: #fff !important; -} - -a.bg-secondary-light:hover, a.bg-secondary-light:focus, -button.bg-secondary-light:hover, -button.bg-secondary-light:focus { - background-color: #e6e6e6 !important; -} - -.bg-tertiary { - background-color: #257af4 !important; -} - -a.bg-tertiary:hover, a.bg-tertiary:focus, -button.bg-tertiary:hover, -button.bg-tertiary:focus { - background-color: #0b60db !important; -} - -.bg-tertiary-light { - background-color: #e3f1fe !important; -} - -a.bg-tertiary-light:hover, a.bg-tertiary-light:focus, -button.bg-tertiary-light:hover, -button.bg-tertiary-light:focus { - background-color: #b2d8fc !important; -} - -.bg-white { - background-color: #fff !important; -} - -a.bg-white:hover, a.bg-white:focus, -button.bg-white:hover, -button.bg-white:focus { - background-color: #e6e6e6 !important; -} - -.bg-black { - background-color: #212529 !important; -} - -a.bg-black:hover, a.bg-black:focus, -button.bg-black:hover, -button.bg-black:focus { - background-color: #0a0c0d !important; -} - -.bg-blue { - background-color: #257af4 !important; -} - -a.bg-blue:hover, a.bg-blue:focus, -button.bg-blue:hover, -button.bg-blue:focus { - background-color: #0b60db !important; -} - -.bg-light-blue { - background-color: #e3f1fe !important; -} - -a.bg-light-blue:hover, a.bg-light-blue:focus, -button.bg-light-blue:hover, -button.bg-light-blue:focus { - background-color: #b2d8fc !important; -} - -.bg-yellow { - background-color: #ffcc00 !important; -} - -a.bg-yellow:hover, a.bg-yellow:focus, -button.bg-yellow:hover, -button.bg-yellow:focus { - background-color: #cca300 !important; -} - -.bg-light-yellow { - background-color: #fffaf0 !important; -} - -a.bg-light-yellow:hover, a.bg-light-yellow:focus, -button.bg-light-yellow:hover, -button.bg-light-yellow:focus { - background-color: #ffe9bd !important; -} - -.bg-orange { - background-color: #ff8c00 !important; -} - -a.bg-orange:hover, a.bg-orange:focus, -button.bg-orange:hover, -button.bg-orange:focus { - background-color: #cc7000 !important; -} - -.bg-light-orange { - background-color: #ffe4b5 !important; -} - -a.bg-light-orange:hover, a.bg-light-orange:focus, -button.bg-light-orange:hover, -button.bg-light-orange:focus { - background-color: #ffd182 !important; -} - -.bg-red { - background-color: #ff3939 !important; -} - -a.bg-red:hover, a.bg-red:focus, -button.bg-red:hover, -button.bg-red:focus { - background-color: #ff0606 !important; -} - -.bg-light-red { - background-color: #ffe4e1 !important; -} - -a.bg-light-red:hover, a.bg-light-red:focus, -button.bg-light-red:hover, -button.bg-light-red:focus { - background-color: #ffb6ae !important; -} - -.bg-medium { - background-color: #d6dbdf !important; -} - -a.bg-medium:hover, a.bg-medium:focus, -button.bg-medium:hover, -button.bg-medium:focus { - background-color: #b9c2c9 !important; -} - -.bg-white { - background-color: #fff !important; -} - -.bg-transparent { - background-color: transparent !important; -} - -.border { - border: 1px solid #d6dbdf !important; -} - -.border-top { - border-top: 1px solid #d6dbdf !important; -} - -.border-right { - border-right: 1px solid #d6dbdf !important; -} - -.border-bottom { - border-bottom: 1px solid #d6dbdf !important; -} - -.border-left { - border-left: 1px solid #d6dbdf !important; -} - -.border-0 { - border: 0 !important; -} - -.border-top-0 { - border-top: 0 !important; -} - -.border-right-0 { - border-right: 0 !important; -} - -.border-bottom-0 { - border-bottom: 0 !important; -} - -.border-left-0 { - border-left: 0 !important; -} - -.border-primary { - border-color: #ffcc00 !important; -} - -.border-secondary { - border-color: #212529 !important; -} - -.border-success { - border-color: #28a745 !important; -} - -.border-info { - border-color: #17a2b8 !important; -} - -.border-warning { - border-color: #ffc107 !important; -} - -.border-danger { - border-color: #dc3545 !important; -} - -.border-light { - border-color: #f1f6f9 !important; -} - -.border-dark { - border-color: #495057 !important; -} - -.border-primary-light { - border-color: #fffaf0 !important; -} - -.border-secondary-light { - border-color: #fff !important; -} - -.border-tertiary { - border-color: #257af4 !important; -} - -.border-tertiary-light { - border-color: #e3f1fe !important; -} - -.border-white { - border-color: #fff !important; -} - -.border-black { - border-color: #212529 !important; -} - -.border-blue { - border-color: #257af4 !important; -} - -.border-light-blue { - border-color: #e3f1fe !important; -} - -.border-yellow { - border-color: #ffcc00 !important; -} - -.border-light-yellow { - border-color: #fffaf0 !important; -} - -.border-orange { - border-color: #ff8c00 !important; -} - -.border-light-orange { - border-color: #ffe4b5 !important; -} - -.border-red { - border-color: #ff3939 !important; -} - -.border-light-red { - border-color: #ffe4e1 !important; -} - -.border-medium { - border-color: #d6dbdf !important; -} - -.border-white { - border-color: #fff !important; -} - -.rounded-sm { - border-radius: 8px !important; -} - -.rounded { - border-radius: 8px !important; -} - -.rounded-top { - border-top-left-radius: 8px !important; - border-top-right-radius: 8px !important; -} - -.rounded-right { - border-top-right-radius: 8px !important; - border-bottom-right-radius: 8px !important; -} - -.rounded-bottom { - border-bottom-right-radius: 8px !important; - border-bottom-left-radius: 8px !important; -} - -.rounded-left { - border-top-left-radius: 8px !important; - border-bottom-left-radius: 8px !important; -} - -.rounded-lg { - border-radius: 8px !important; -} - -.rounded-circle { - border-radius: 50% !important; -} - -.rounded-pill { - border-radius: 50rem !important; -} - -.rounded-0 { - border-radius: 0 !important; -} - -.clearfix::after { - display: block; - clear: both; - content: ""; -} - -.d-none { - display: none !important; -} - -.d-inline { - display: inline !important; -} - -.d-inline-block { - display: inline-block !important; -} - -.d-block { - display: block !important; -} - -.d-table { - display: table !important; -} - -.d-table-row { - display: table-row !important; -} - -.d-table-cell { - display: table-cell !important; -} - -.d-flex { - display: flex !important; -} - -.d-inline-flex { - display: inline-flex !important; -} - -@media (min-width: 400px) { - .d-xs-none { - display: none !important; - } - - .d-xs-inline { - display: inline !important; - } - - .d-xs-inline-block { - display: inline-block !important; - } - - .d-xs-block { - display: block !important; - } - - .d-xs-table { - display: table !important; - } - - .d-xs-table-row { - display: table-row !important; - } - - .d-xs-table-cell { - display: table-cell !important; - } - - .d-xs-flex { - display: flex !important; - } - - .d-xs-inline-flex { - display: inline-flex !important; - } -} -@media (min-width: 616px) { - .d-sm-none { - display: none !important; - } - - .d-sm-inline { - display: inline !important; - } - - .d-sm-inline-block { - display: inline-block !important; - } - - .d-sm-block { - display: block !important; - } - - .d-sm-table { - display: table !important; - } - - .d-sm-table-row { - display: table-row !important; - } - - .d-sm-table-cell { - display: table-cell !important; - } - - .d-sm-flex { - display: flex !important; - } - - .d-sm-inline-flex { - display: inline-flex !important; - } -} -@media (min-width: 768px) { - .d-md-none { - display: none !important; - } - - .d-md-inline { - display: inline !important; - } - - .d-md-inline-block { - display: inline-block !important; - } - - .d-md-block { - display: block !important; - } - - .d-md-table { - display: table !important; - } - - .d-md-table-row { - display: table-row !important; - } - - .d-md-table-cell { - display: table-cell !important; - } - - .d-md-flex { - display: flex !important; - } - - .d-md-inline-flex { - display: inline-flex !important; - } -} -@media (min-width: 980px) { - .d-lg-none { - display: none !important; - } - - .d-lg-inline { - display: inline !important; - } - - .d-lg-inline-block { - display: inline-block !important; - } - - .d-lg-block { - display: block !important; - } - - .d-lg-table { - display: table !important; - } - - .d-lg-table-row { - display: table-row !important; - } - - .d-lg-table-cell { - display: table-cell !important; - } - - .d-lg-flex { - display: flex !important; - } - - .d-lg-inline-flex { - display: inline-flex !important; - } -} -@media (min-width: 1240px) { - .d-xl-none { - display: none !important; - } - - .d-xl-inline { - display: inline !important; - } - - .d-xl-inline-block { - display: inline-block !important; - } - - .d-xl-block { - display: block !important; - } - - .d-xl-table { - display: table !important; - } - - .d-xl-table-row { - display: table-row !important; - } - - .d-xl-table-cell { - display: table-cell !important; - } - - .d-xl-flex { - display: flex !important; - } - - .d-xl-inline-flex { - display: inline-flex !important; - } -} -@media print { - .d-print-none { - display: none !important; - } - - .d-print-inline { - display: inline !important; - } - - .d-print-inline-block { - display: inline-block !important; - } - - .d-print-block { - display: block !important; - } - - .d-print-table { - display: table !important; - } - - .d-print-table-row { - display: table-row !important; - } - - .d-print-table-cell { - display: table-cell !important; - } - - .d-print-flex { - display: flex !important; - } - - .d-print-inline-flex { - display: inline-flex !important; - } -} -.embed-responsive { - position: relative; - display: block; - width: 100%; - padding: 0; - overflow: hidden; -} -.embed-responsive::before { - display: block; - content: ""; -} -.embed-responsive .embed-responsive-item, -.embed-responsive iframe, -.embed-responsive embed, -.embed-responsive object, -.embed-responsive video { - position: absolute; - top: 0; - bottom: 0; - left: 0; - width: 100%; - height: 100%; - border: 0; -} - -.embed-responsive-21by9::before { - padding-top: 42.8571428571%; -} - -.embed-responsive-16by9::before { - padding-top: 56.25%; -} - -.embed-responsive-4by3::before { - padding-top: 75%; -} - -.embed-responsive-1by1::before { - padding-top: 100%; -} - -.flex-row { - flex-direction: row !important; -} - -.flex-column { - flex-direction: column !important; -} - -.flex-row-reverse { - flex-direction: row-reverse !important; -} - -.flex-column-reverse { - flex-direction: column-reverse !important; -} - -.flex-wrap { - flex-wrap: wrap !important; -} - -.flex-nowrap { - flex-wrap: nowrap !important; -} - -.flex-wrap-reverse { - flex-wrap: wrap-reverse !important; -} - -.flex-fill { - flex: 1 1 auto !important; -} - -.flex-grow-0 { - flex-grow: 0 !important; -} - -.flex-grow-1 { - flex-grow: 1 !important; -} - -.flex-shrink-0 { - flex-shrink: 0 !important; -} - -.flex-shrink-1 { - flex-shrink: 1 !important; -} - -.justify-content-start { - justify-content: flex-start !important; -} - -.justify-content-end { - justify-content: flex-end !important; -} - -.justify-content-center { - justify-content: center !important; -} - -.justify-content-between { - justify-content: space-between !important; -} - -.justify-content-around { - justify-content: space-around !important; -} - -.align-items-start { - align-items: flex-start !important; -} - -.align-items-end { - align-items: flex-end !important; -} - -.align-items-center { - align-items: center !important; -} - -.align-items-baseline { - align-items: baseline !important; -} - -.align-items-stretch { - align-items: stretch !important; -} - -.align-content-start { - align-content: flex-start !important; -} - -.align-content-end { - align-content: flex-end !important; -} - -.align-content-center { - align-content: center !important; -} - -.align-content-between { - align-content: space-between !important; -} - -.align-content-around { - align-content: space-around !important; -} - -.align-content-stretch { - align-content: stretch !important; -} - -.align-self-auto { - align-self: auto !important; -} - -.align-self-start { - align-self: flex-start !important; -} - -.align-self-end { - align-self: flex-end !important; -} - -.align-self-center { - align-self: center !important; -} - -.align-self-baseline { - align-self: baseline !important; -} - -.align-self-stretch { - align-self: stretch !important; -} - -@media (min-width: 400px) { - .flex-xs-row { - flex-direction: row !important; - } - - .flex-xs-column { - flex-direction: column !important; - } - - .flex-xs-row-reverse { - flex-direction: row-reverse !important; - } - - .flex-xs-column-reverse { - flex-direction: column-reverse !important; - } - - .flex-xs-wrap { - flex-wrap: wrap !important; - } - - .flex-xs-nowrap { - flex-wrap: nowrap !important; - } - - .flex-xs-wrap-reverse { - flex-wrap: wrap-reverse !important; - } - - .flex-xs-fill { - flex: 1 1 auto !important; - } - - .flex-xs-grow-0 { - flex-grow: 0 !important; - } - - .flex-xs-grow-1 { - flex-grow: 1 !important; - } - - .flex-xs-shrink-0 { - flex-shrink: 0 !important; - } - - .flex-xs-shrink-1 { - flex-shrink: 1 !important; - } - - .justify-content-xs-start { - justify-content: flex-start !important; - } - - .justify-content-xs-end { - justify-content: flex-end !important; - } - - .justify-content-xs-center { - justify-content: center !important; - } - - .justify-content-xs-between { - justify-content: space-between !important; - } - - .justify-content-xs-around { - justify-content: space-around !important; - } - - .align-items-xs-start { - align-items: flex-start !important; - } - - .align-items-xs-end { - align-items: flex-end !important; - } - - .align-items-xs-center { - align-items: center !important; - } - - .align-items-xs-baseline { - align-items: baseline !important; - } - - .align-items-xs-stretch { - align-items: stretch !important; - } - - .align-content-xs-start { - align-content: flex-start !important; - } - - .align-content-xs-end { - align-content: flex-end !important; - } - - .align-content-xs-center { - align-content: center !important; - } - - .align-content-xs-between { - align-content: space-between !important; - } - - .align-content-xs-around { - align-content: space-around !important; - } - - .align-content-xs-stretch { - align-content: stretch !important; - } - - .align-self-xs-auto { - align-self: auto !important; - } - - .align-self-xs-start { - align-self: flex-start !important; - } - - .align-self-xs-end { - align-self: flex-end !important; - } - - .align-self-xs-center { - align-self: center !important; - } - - .align-self-xs-baseline { - align-self: baseline !important; - } - - .align-self-xs-stretch { - align-self: stretch !important; - } -} -@media (min-width: 616px) { - .flex-sm-row { - flex-direction: row !important; - } - - .flex-sm-column { - flex-direction: column !important; - } - - .flex-sm-row-reverse { - flex-direction: row-reverse !important; - } - - .flex-sm-column-reverse { - flex-direction: column-reverse !important; - } - - .flex-sm-wrap { - flex-wrap: wrap !important; - } - - .flex-sm-nowrap { - flex-wrap: nowrap !important; - } - - .flex-sm-wrap-reverse { - flex-wrap: wrap-reverse !important; - } - - .flex-sm-fill { - flex: 1 1 auto !important; - } - - .flex-sm-grow-0 { - flex-grow: 0 !important; - } - - .flex-sm-grow-1 { - flex-grow: 1 !important; - } - - .flex-sm-shrink-0 { - flex-shrink: 0 !important; - } - - .flex-sm-shrink-1 { - flex-shrink: 1 !important; - } - - .justify-content-sm-start { - justify-content: flex-start !important; - } - - .justify-content-sm-end { - justify-content: flex-end !important; - } - - .justify-content-sm-center { - justify-content: center !important; - } - - .justify-content-sm-between { - justify-content: space-between !important; - } - - .justify-content-sm-around { - justify-content: space-around !important; - } - - .align-items-sm-start { - align-items: flex-start !important; - } - - .align-items-sm-end { - align-items: flex-end !important; - } - - .align-items-sm-center { - align-items: center !important; - } - - .align-items-sm-baseline { - align-items: baseline !important; - } - - .align-items-sm-stretch { - align-items: stretch !important; - } - - .align-content-sm-start { - align-content: flex-start !important; - } - - .align-content-sm-end { - align-content: flex-end !important; - } - - .align-content-sm-center { - align-content: center !important; - } - - .align-content-sm-between { - align-content: space-between !important; - } - - .align-content-sm-around { - align-content: space-around !important; - } - - .align-content-sm-stretch { - align-content: stretch !important; - } - - .align-self-sm-auto { - align-self: auto !important; - } - - .align-self-sm-start { - align-self: flex-start !important; - } - - .align-self-sm-end { - align-self: flex-end !important; - } - - .align-self-sm-center { - align-self: center !important; - } - - .align-self-sm-baseline { - align-self: baseline !important; - } - - .align-self-sm-stretch { - align-self: stretch !important; - } -} -@media (min-width: 768px) { - .flex-md-row { - flex-direction: row !important; - } - - .flex-md-column { - flex-direction: column !important; - } - - .flex-md-row-reverse { - flex-direction: row-reverse !important; - } - - .flex-md-column-reverse { - flex-direction: column-reverse !important; - } - - .flex-md-wrap { - flex-wrap: wrap !important; - } - - .flex-md-nowrap { - flex-wrap: nowrap !important; - } - - .flex-md-wrap-reverse { - flex-wrap: wrap-reverse !important; - } - - .flex-md-fill { - flex: 1 1 auto !important; - } - - .flex-md-grow-0 { - flex-grow: 0 !important; - } - - .flex-md-grow-1 { - flex-grow: 1 !important; - } - - .flex-md-shrink-0 { - flex-shrink: 0 !important; - } - - .flex-md-shrink-1 { - flex-shrink: 1 !important; - } - - .justify-content-md-start { - justify-content: flex-start !important; - } - - .justify-content-md-end { - justify-content: flex-end !important; - } - - .justify-content-md-center { - justify-content: center !important; - } - - .justify-content-md-between { - justify-content: space-between !important; - } - - .justify-content-md-around { - justify-content: space-around !important; - } - - .align-items-md-start { - align-items: flex-start !important; - } - - .align-items-md-end { - align-items: flex-end !important; - } - - .align-items-md-center { - align-items: center !important; - } - - .align-items-md-baseline { - align-items: baseline !important; - } - - .align-items-md-stretch { - align-items: stretch !important; - } - - .align-content-md-start { - align-content: flex-start !important; - } - - .align-content-md-end { - align-content: flex-end !important; - } - - .align-content-md-center { - align-content: center !important; - } - - .align-content-md-between { - align-content: space-between !important; - } - - .align-content-md-around { - align-content: space-around !important; - } - - .align-content-md-stretch { - align-content: stretch !important; - } - - .align-self-md-auto { - align-self: auto !important; - } - - .align-self-md-start { - align-self: flex-start !important; - } - - .align-self-md-end { - align-self: flex-end !important; - } - - .align-self-md-center { - align-self: center !important; - } - - .align-self-md-baseline { - align-self: baseline !important; - } - - .align-self-md-stretch { - align-self: stretch !important; - } -} -@media (min-width: 980px) { - .flex-lg-row { - flex-direction: row !important; - } - - .flex-lg-column { - flex-direction: column !important; - } - - .flex-lg-row-reverse { - flex-direction: row-reverse !important; - } - - .flex-lg-column-reverse { - flex-direction: column-reverse !important; - } - - .flex-lg-wrap { - flex-wrap: wrap !important; - } - - .flex-lg-nowrap { - flex-wrap: nowrap !important; - } - - .flex-lg-wrap-reverse { - flex-wrap: wrap-reverse !important; - } - - .flex-lg-fill { - flex: 1 1 auto !important; - } - - .flex-lg-grow-0 { - flex-grow: 0 !important; - } - - .flex-lg-grow-1 { - flex-grow: 1 !important; - } - - .flex-lg-shrink-0 { - flex-shrink: 0 !important; - } - - .flex-lg-shrink-1 { - flex-shrink: 1 !important; - } - - .justify-content-lg-start { - justify-content: flex-start !important; - } - - .justify-content-lg-end { - justify-content: flex-end !important; - } - - .justify-content-lg-center { - justify-content: center !important; - } - - .justify-content-lg-between { - justify-content: space-between !important; - } - - .justify-content-lg-around { - justify-content: space-around !important; - } - - .align-items-lg-start { - align-items: flex-start !important; - } - - .align-items-lg-end { - align-items: flex-end !important; - } - - .align-items-lg-center { - align-items: center !important; - } - - .align-items-lg-baseline { - align-items: baseline !important; - } - - .align-items-lg-stretch { - align-items: stretch !important; - } - - .align-content-lg-start { - align-content: flex-start !important; - } - - .align-content-lg-end { - align-content: flex-end !important; - } - - .align-content-lg-center { - align-content: center !important; - } - - .align-content-lg-between { - align-content: space-between !important; - } - - .align-content-lg-around { - align-content: space-around !important; - } - - .align-content-lg-stretch { - align-content: stretch !important; - } - - .align-self-lg-auto { - align-self: auto !important; - } - - .align-self-lg-start { - align-self: flex-start !important; - } - - .align-self-lg-end { - align-self: flex-end !important; - } - - .align-self-lg-center { - align-self: center !important; - } - - .align-self-lg-baseline { - align-self: baseline !important; - } - - .align-self-lg-stretch { - align-self: stretch !important; - } -} -@media (min-width: 1240px) { - .flex-xl-row { - flex-direction: row !important; - } - - .flex-xl-column { - flex-direction: column !important; - } - - .flex-xl-row-reverse { - flex-direction: row-reverse !important; - } - - .flex-xl-column-reverse { - flex-direction: column-reverse !important; - } - - .flex-xl-wrap { - flex-wrap: wrap !important; - } - - .flex-xl-nowrap { - flex-wrap: nowrap !important; - } - - .flex-xl-wrap-reverse { - flex-wrap: wrap-reverse !important; - } - - .flex-xl-fill { - flex: 1 1 auto !important; - } - - .flex-xl-grow-0 { - flex-grow: 0 !important; - } - - .flex-xl-grow-1 { - flex-grow: 1 !important; - } - - .flex-xl-shrink-0 { - flex-shrink: 0 !important; - } - - .flex-xl-shrink-1 { - flex-shrink: 1 !important; - } - - .justify-content-xl-start { - justify-content: flex-start !important; - } - - .justify-content-xl-end { - justify-content: flex-end !important; - } - - .justify-content-xl-center { - justify-content: center !important; - } - - .justify-content-xl-between { - justify-content: space-between !important; - } - - .justify-content-xl-around { - justify-content: space-around !important; - } - - .align-items-xl-start { - align-items: flex-start !important; - } - - .align-items-xl-end { - align-items: flex-end !important; - } - - .align-items-xl-center { - align-items: center !important; - } - - .align-items-xl-baseline { - align-items: baseline !important; - } - - .align-items-xl-stretch { - align-items: stretch !important; - } - - .align-content-xl-start { - align-content: flex-start !important; - } - - .align-content-xl-end { - align-content: flex-end !important; - } - - .align-content-xl-center { - align-content: center !important; - } - - .align-content-xl-between { - align-content: space-between !important; - } - - .align-content-xl-around { - align-content: space-around !important; - } - - .align-content-xl-stretch { - align-content: stretch !important; - } - - .align-self-xl-auto { - align-self: auto !important; - } - - .align-self-xl-start { - align-self: flex-start !important; - } - - .align-self-xl-end { - align-self: flex-end !important; - } - - .align-self-xl-center { - align-self: center !important; - } - - .align-self-xl-baseline { - align-self: baseline !important; - } - - .align-self-xl-stretch { - align-self: stretch !important; - } -} -.float-left { - float: left !important; -} - -.float-right { - float: right !important; -} - -.float-none { - float: none !important; -} - -@media (min-width: 400px) { - .float-xs-left { - float: left !important; - } - - .float-xs-right { - float: right !important; - } - - .float-xs-none { - float: none !important; - } -} -@media (min-width: 616px) { - .float-sm-left { - float: left !important; - } - - .float-sm-right { - float: right !important; - } - - .float-sm-none { - float: none !important; - } -} -@media (min-width: 768px) { - .float-md-left { - float: left !important; - } - - .float-md-right { - float: right !important; - } - - .float-md-none { - float: none !important; - } -} -@media (min-width: 980px) { - .float-lg-left { - float: left !important; - } - - .float-lg-right { - float: right !important; - } - - .float-lg-none { - float: none !important; - } -} -@media (min-width: 1240px) { - .float-xl-left { - float: left !important; - } - - .float-xl-right { - float: right !important; - } - - .float-xl-none { - float: none !important; - } -} -.overflow-auto { - overflow: auto !important; -} - -.overflow-hidden { - overflow: hidden !important; -} - -.position-static { - position: static !important; -} - -.position-relative { - position: relative !important; -} - -.position-absolute { - position: absolute !important; -} - -.position-fixed { - position: fixed !important; -} - -.position-sticky { - position: sticky !important; -} - -.fixed-top { - position: fixed; - top: 0; - right: 0; - left: 0; - z-index: 1030; -} - -.fixed-bottom { - position: fixed; - right: 0; - bottom: 0; - left: 0; - z-index: 1030; -} - -@supports (position: sticky) { - .sticky-top { - position: sticky; - top: 0; - z-index: 1020; - } -} - -.sr-only { - position: absolute; - width: 1px; - height: 1px; - padding: 0; - margin: -1px; - overflow: hidden; - clip: rect(0, 0, 0, 0); - white-space: nowrap; - border: 0; -} - -.sr-only-focusable:active, .sr-only-focusable:focus { - position: static; - width: auto; - height: auto; - overflow: visible; - clip: auto; - white-space: normal; -} - -.shadow-sm { - box-shadow: 0 2px 14px rgba(108, 117, 125, 0.2) !important; -} - -.shadow { - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2) !important; -} - -.shadow-lg { - box-shadow: 0 12px 32px rgba(108, 117, 125, 0.2) !important; -} - -.shadow-none { - box-shadow: none !important; -} - -.w-25 { - width: 25% !important; -} - -.w-50 { - width: 50% !important; -} - -.w-75 { - width: 75% !important; -} - -.w-100 { - width: 100% !important; -} - -.w-auto { - width: auto !important; -} - -.h-25 { - height: 25% !important; -} - -.h-50 { - height: 50% !important; -} - -.h-75 { - height: 75% !important; -} - -.h-100 { - height: 100% !important; -} - -.h-auto { - height: auto !important; -} - -.mw-100 { - max-width: 100% !important; -} - -.mh-100 { - max-height: 100% !important; -} - -.min-vw-100 { - min-width: 100vw !important; -} - -.min-vh-100 { - min-height: 100vh !important; -} - -.vw-100 { - width: 100vw !important; -} - -.vh-100 { - height: 100vh !important; -} - -.stretched-link::after { - position: absolute; - top: 0; - right: 0; - bottom: 0; - left: 0; - z-index: 1; - pointer-events: auto; - content: ""; - background-color: rgba(0, 0, 0, 0); -} - -.m-0 { - margin: 0 !important; -} - -.mt-0, -.my-0 { - margin-top: 0 !important; -} - -.mr-0, -.mx-0 { - margin-right: 0 !important; -} - -.mb-0, -.my-0 { - margin-bottom: 0 !important; -} - -.ml-0, -.mx-0 { - margin-left: 0 !important; -} - -.m-1 { - margin: 8px !important; -} - -.mt-1, -.my-1 { - margin-top: 8px !important; -} - -.mr-1, -.mx-1 { - margin-right: 8px !important; -} - -.mb-1, -.my-1 { - margin-bottom: 8px !important; -} - -.ml-1, -.mx-1 { - margin-left: 8px !important; -} - -.m-2 { - margin: 16px !important; -} - -.mt-2, -.my-2 { - margin-top: 16px !important; -} - -.mr-2, -.mx-2 { - margin-right: 16px !important; -} - -.mb-2, -.my-2 { - margin-bottom: 16px !important; -} - -.ml-2, -.mx-2 { - margin-left: 16px !important; -} - -.m-3 { - margin: 24px !important; -} - -.mt-3, -.my-3 { - margin-top: 24px !important; -} - -.mr-3, -.mx-3 { - margin-right: 24px !important; -} - -.mb-3, -.my-3 { - margin-bottom: 24px !important; -} - -.ml-3, -.mx-3 { - margin-left: 24px !important; -} - -.m-4 { - margin: 32px !important; -} - -.mt-4, -.my-4 { - margin-top: 32px !important; -} - -.mr-4, -.mx-4 { - margin-right: 32px !important; -} - -.mb-4, -.my-4 { - margin-bottom: 32px !important; -} - -.ml-4, -.mx-4 { - margin-left: 32px !important; -} - -.m-5 { - margin: 40px !important; -} - -.mt-5, -.my-5 { - margin-top: 40px !important; -} - -.mr-5, -.mx-5 { - margin-right: 40px !important; -} - -.mb-5, -.my-5 { - margin-bottom: 40px !important; -} - -.ml-5, -.mx-5 { - margin-left: 40px !important; -} - -.m-6 { - margin: 48px !important; -} - -.mt-6, -.my-6 { - margin-top: 48px !important; -} - -.mr-6, -.mx-6 { - margin-right: 48px !important; -} - -.mb-6, -.my-6 { - margin-bottom: 48px !important; -} - -.ml-6, -.mx-6 { - margin-left: 48px !important; -} - -.m-7 { - margin: 56px !important; -} - -.mt-7, -.my-7 { - margin-top: 56px !important; -} - -.mr-7, -.mx-7 { - margin-right: 56px !important; -} - -.mb-7, -.my-7 { - margin-bottom: 56px !important; -} - -.ml-7, -.mx-7 { - margin-left: 56px !important; -} - -.m-8 { - margin: 64px !important; -} - -.mt-8, -.my-8 { - margin-top: 64px !important; -} - -.mr-8, -.mx-8 { - margin-right: 64px !important; -} - -.mb-8, -.my-8 { - margin-bottom: 64px !important; -} - -.ml-8, -.mx-8 { - margin-left: 64px !important; -} - -.m-9 { - margin: 72px !important; -} - -.mt-9, -.my-9 { - margin-top: 72px !important; -} - -.mr-9, -.mx-9 { - margin-right: 72px !important; -} - -.mb-9, -.my-9 { - margin-bottom: 72px !important; -} - -.ml-9, -.mx-9 { - margin-left: 72px !important; -} - -.m-10 { - margin: 80px !important; -} - -.mt-10, -.my-10 { - margin-top: 80px !important; -} - -.mr-10, -.mx-10 { - margin-right: 80px !important; -} - -.mb-10, -.my-10 { - margin-bottom: 80px !important; -} - -.ml-10, -.mx-10 { - margin-left: 80px !important; -} - -.m-12 { - margin: 96px !important; -} - -.mt-12, -.my-12 { - margin-top: 96px !important; -} - -.mr-12, -.mx-12 { - margin-right: 96px !important; -} - -.mb-12, -.my-12 { - margin-bottom: 96px !important; -} - -.ml-12, -.mx-12 { - margin-left: 96px !important; -} - -.m-15 { - margin: 120px !important; -} - -.mt-15, -.my-15 { - margin-top: 120px !important; -} - -.mr-15, -.mx-15 { - margin-right: 120px !important; -} - -.mb-15, -.my-15 { - margin-bottom: 120px !important; -} - -.ml-15, -.mx-15 { - margin-left: 120px !important; -} - -.p-0 { - padding: 0 !important; -} - -.pt-0, -.py-0 { - padding-top: 0 !important; -} - -.pr-0, -.px-0 { - padding-right: 0 !important; -} - -.pb-0, -.py-0 { - padding-bottom: 0 !important; -} - -.pl-0, -.px-0 { - padding-left: 0 !important; -} - -.p-1 { - padding: 8px !important; -} - -.pt-1, -.py-1 { - padding-top: 8px !important; -} - -.pr-1, -.px-1 { - padding-right: 8px !important; -} - -.pb-1, -.py-1 { - padding-bottom: 8px !important; -} - -.pl-1, -.px-1 { - padding-left: 8px !important; -} - -.p-2 { - padding: 16px !important; -} - -.pt-2, -.py-2 { - padding-top: 16px !important; -} - -.pr-2, -.px-2 { - padding-right: 16px !important; -} - -.pb-2, -.py-2 { - padding-bottom: 16px !important; -} - -.pl-2, -.px-2 { - padding-left: 16px !important; -} - -.p-3 { - padding: 24px !important; -} - -.pt-3, -.py-3 { - padding-top: 24px !important; -} - -.pr-3, -.px-3 { - padding-right: 24px !important; -} - -.pb-3, -.py-3 { - padding-bottom: 24px !important; -} - -.pl-3, -.px-3 { - padding-left: 24px !important; -} - -.p-4 { - padding: 32px !important; -} - -.pt-4, -.py-4 { - padding-top: 32px !important; -} - -.pr-4, -.px-4 { - padding-right: 32px !important; -} - -.pb-4, -.py-4 { - padding-bottom: 32px !important; -} - -.pl-4, -.px-4 { - padding-left: 32px !important; -} - -.p-5 { - padding: 40px !important; -} - -.pt-5, -.py-5 { - padding-top: 40px !important; -} - -.pr-5, -.px-5 { - padding-right: 40px !important; -} - -.pb-5, -.py-5 { - padding-bottom: 40px !important; -} - -.pl-5, -.px-5 { - padding-left: 40px !important; -} - -.p-6 { - padding: 48px !important; -} - -.pt-6, -.py-6 { - padding-top: 48px !important; -} - -.pr-6, -.px-6 { - padding-right: 48px !important; -} - -.pb-6, -.py-6 { - padding-bottom: 48px !important; -} - -.pl-6, -.px-6 { - padding-left: 48px !important; -} - -.p-7 { - padding: 56px !important; -} - -.pt-7, -.py-7 { - padding-top: 56px !important; -} - -.pr-7, -.px-7 { - padding-right: 56px !important; -} - -.pb-7, -.py-7 { - padding-bottom: 56px !important; -} - -.pl-7, -.px-7 { - padding-left: 56px !important; -} - -.p-8 { - padding: 64px !important; -} - -.pt-8, -.py-8 { - padding-top: 64px !important; -} - -.pr-8, -.px-8 { - padding-right: 64px !important; -} - -.pb-8, -.py-8 { - padding-bottom: 64px !important; -} - -.pl-8, -.px-8 { - padding-left: 64px !important; -} - -.p-9 { - padding: 72px !important; -} - -.pt-9, -.py-9 { - padding-top: 72px !important; -} - -.pr-9, -.px-9 { - padding-right: 72px !important; -} - -.pb-9, -.py-9 { - padding-bottom: 72px !important; -} - -.pl-9, -.px-9 { - padding-left: 72px !important; -} - -.p-10 { - padding: 80px !important; -} - -.pt-10, -.py-10 { - padding-top: 80px !important; -} - -.pr-10, -.px-10 { - padding-right: 80px !important; -} - -.pb-10, -.py-10 { - padding-bottom: 80px !important; -} - -.pl-10, -.px-10 { - padding-left: 80px !important; -} - -.p-12 { - padding: 96px !important; -} - -.pt-12, -.py-12 { - padding-top: 96px !important; -} - -.pr-12, -.px-12 { - padding-right: 96px !important; -} - -.pb-12, -.py-12 { - padding-bottom: 96px !important; -} - -.pl-12, -.px-12 { - padding-left: 96px !important; -} - -.p-15 { - padding: 120px !important; -} - -.pt-15, -.py-15 { - padding-top: 120px !important; -} - -.pr-15, -.px-15 { - padding-right: 120px !important; -} - -.pb-15, -.py-15 { - padding-bottom: 120px !important; -} - -.pl-15, -.px-15 { - padding-left: 120px !important; -} - -.m-n1 { - margin: -8px !important; -} - -.mt-n1, -.my-n1 { - margin-top: -8px !important; -} - -.mr-n1, -.mx-n1 { - margin-right: -8px !important; -} - -.mb-n1, -.my-n1 { - margin-bottom: -8px !important; -} - -.ml-n1, -.mx-n1 { - margin-left: -8px !important; -} - -.m-n2 { - margin: -16px !important; -} - -.mt-n2, -.my-n2 { - margin-top: -16px !important; -} - -.mr-n2, -.mx-n2 { - margin-right: -16px !important; -} - -.mb-n2, -.my-n2 { - margin-bottom: -16px !important; -} - -.ml-n2, -.mx-n2 { - margin-left: -16px !important; -} - -.m-n3 { - margin: -24px !important; -} - -.mt-n3, -.my-n3 { - margin-top: -24px !important; -} - -.mr-n3, -.mx-n3 { - margin-right: -24px !important; -} - -.mb-n3, -.my-n3 { - margin-bottom: -24px !important; -} - -.ml-n3, -.mx-n3 { - margin-left: -24px !important; -} - -.m-n4 { - margin: -32px !important; -} - -.mt-n4, -.my-n4 { - margin-top: -32px !important; -} - -.mr-n4, -.mx-n4 { - margin-right: -32px !important; -} - -.mb-n4, -.my-n4 { - margin-bottom: -32px !important; -} - -.ml-n4, -.mx-n4 { - margin-left: -32px !important; -} - -.m-n5 { - margin: -40px !important; -} - -.mt-n5, -.my-n5 { - margin-top: -40px !important; -} - -.mr-n5, -.mx-n5 { - margin-right: -40px !important; -} - -.mb-n5, -.my-n5 { - margin-bottom: -40px !important; -} - -.ml-n5, -.mx-n5 { - margin-left: -40px !important; -} - -.m-n6 { - margin: -48px !important; -} - -.mt-n6, -.my-n6 { - margin-top: -48px !important; -} - -.mr-n6, -.mx-n6 { - margin-right: -48px !important; -} - -.mb-n6, -.my-n6 { - margin-bottom: -48px !important; -} - -.ml-n6, -.mx-n6 { - margin-left: -48px !important; -} - -.m-n7 { - margin: -56px !important; -} - -.mt-n7, -.my-n7 { - margin-top: -56px !important; -} - -.mr-n7, -.mx-n7 { - margin-right: -56px !important; -} - -.mb-n7, -.my-n7 { - margin-bottom: -56px !important; -} - -.ml-n7, -.mx-n7 { - margin-left: -56px !important; -} - -.m-n8 { - margin: -64px !important; -} - -.mt-n8, -.my-n8 { - margin-top: -64px !important; -} - -.mr-n8, -.mx-n8 { - margin-right: -64px !important; -} - -.mb-n8, -.my-n8 { - margin-bottom: -64px !important; -} - -.ml-n8, -.mx-n8 { - margin-left: -64px !important; -} - -.m-n9 { - margin: -72px !important; -} - -.mt-n9, -.my-n9 { - margin-top: -72px !important; -} - -.mr-n9, -.mx-n9 { - margin-right: -72px !important; -} - -.mb-n9, -.my-n9 { - margin-bottom: -72px !important; -} - -.ml-n9, -.mx-n9 { - margin-left: -72px !important; -} - -.m-n10 { - margin: -80px !important; -} - -.mt-n10, -.my-n10 { - margin-top: -80px !important; -} - -.mr-n10, -.mx-n10 { - margin-right: -80px !important; -} - -.mb-n10, -.my-n10 { - margin-bottom: -80px !important; -} - -.ml-n10, -.mx-n10 { - margin-left: -80px !important; -} - -.m-n12 { - margin: -96px !important; -} - -.mt-n12, -.my-n12 { - margin-top: -96px !important; -} - -.mr-n12, -.mx-n12 { - margin-right: -96px !important; -} - -.mb-n12, -.my-n12 { - margin-bottom: -96px !important; -} - -.ml-n12, -.mx-n12 { - margin-left: -96px !important; -} - -.m-n15 { - margin: -120px !important; -} - -.mt-n15, -.my-n15 { - margin-top: -120px !important; -} - -.mr-n15, -.mx-n15 { - margin-right: -120px !important; -} - -.mb-n15, -.my-n15 { - margin-bottom: -120px !important; -} - -.ml-n15, -.mx-n15 { - margin-left: -120px !important; -} - -.m-auto { - margin: auto !important; -} - -.mt-auto, -.my-auto { - margin-top: auto !important; -} - -.mr-auto, -.mx-auto { - margin-right: auto !important; -} - -.mb-auto, -.my-auto { - margin-bottom: auto !important; -} - -.ml-auto, -.mx-auto { - margin-left: auto !important; -} - -@media (min-width: 400px) { - .m-xs-0 { - margin: 0 !important; - } - - .mt-xs-0, -.my-xs-0 { - margin-top: 0 !important; - } - - .mr-xs-0, -.mx-xs-0 { - margin-right: 0 !important; - } - - .mb-xs-0, -.my-xs-0 { - margin-bottom: 0 !important; - } - - .ml-xs-0, -.mx-xs-0 { - margin-left: 0 !important; - } - - .m-xs-1 { - margin: 8px !important; - } - - .mt-xs-1, -.my-xs-1 { - margin-top: 8px !important; - } - - .mr-xs-1, -.mx-xs-1 { - margin-right: 8px !important; - } - - .mb-xs-1, -.my-xs-1 { - margin-bottom: 8px !important; - } - - .ml-xs-1, -.mx-xs-1 { - margin-left: 8px !important; - } - - .m-xs-2 { - margin: 16px !important; - } - - .mt-xs-2, -.my-xs-2 { - margin-top: 16px !important; - } - - .mr-xs-2, -.mx-xs-2 { - margin-right: 16px !important; - } - - .mb-xs-2, -.my-xs-2 { - margin-bottom: 16px !important; - } - - .ml-xs-2, -.mx-xs-2 { - margin-left: 16px !important; - } - - .m-xs-3 { - margin: 24px !important; - } - - .mt-xs-3, -.my-xs-3 { - margin-top: 24px !important; - } - - .mr-xs-3, -.mx-xs-3 { - margin-right: 24px !important; - } - - .mb-xs-3, -.my-xs-3 { - margin-bottom: 24px !important; - } - - .ml-xs-3, -.mx-xs-3 { - margin-left: 24px !important; - } - - .m-xs-4 { - margin: 32px !important; - } - - .mt-xs-4, -.my-xs-4 { - margin-top: 32px !important; - } - - .mr-xs-4, -.mx-xs-4 { - margin-right: 32px !important; - } - - .mb-xs-4, -.my-xs-4 { - margin-bottom: 32px !important; - } - - .ml-xs-4, -.mx-xs-4 { - margin-left: 32px !important; - } - - .m-xs-5 { - margin: 40px !important; - } - - .mt-xs-5, -.my-xs-5 { - margin-top: 40px !important; - } - - .mr-xs-5, -.mx-xs-5 { - margin-right: 40px !important; - } - - .mb-xs-5, -.my-xs-5 { - margin-bottom: 40px !important; - } - - .ml-xs-5, -.mx-xs-5 { - margin-left: 40px !important; - } - - .m-xs-6 { - margin: 48px !important; - } - - .mt-xs-6, -.my-xs-6 { - margin-top: 48px !important; - } - - .mr-xs-6, -.mx-xs-6 { - margin-right: 48px !important; - } - - .mb-xs-6, -.my-xs-6 { - margin-bottom: 48px !important; - } - - .ml-xs-6, -.mx-xs-6 { - margin-left: 48px !important; - } - - .m-xs-7 { - margin: 56px !important; - } - - .mt-xs-7, -.my-xs-7 { - margin-top: 56px !important; - } - - .mr-xs-7, -.mx-xs-7 { - margin-right: 56px !important; - } - - .mb-xs-7, -.my-xs-7 { - margin-bottom: 56px !important; - } - - .ml-xs-7, -.mx-xs-7 { - margin-left: 56px !important; - } - - .m-xs-8 { - margin: 64px !important; - } - - .mt-xs-8, -.my-xs-8 { - margin-top: 64px !important; - } - - .mr-xs-8, -.mx-xs-8 { - margin-right: 64px !important; - } - - .mb-xs-8, -.my-xs-8 { - margin-bottom: 64px !important; - } - - .ml-xs-8, -.mx-xs-8 { - margin-left: 64px !important; - } - - .m-xs-9 { - margin: 72px !important; - } - - .mt-xs-9, -.my-xs-9 { - margin-top: 72px !important; - } - - .mr-xs-9, -.mx-xs-9 { - margin-right: 72px !important; - } - - .mb-xs-9, -.my-xs-9 { - margin-bottom: 72px !important; - } - - .ml-xs-9, -.mx-xs-9 { - margin-left: 72px !important; - } - - .m-xs-10 { - margin: 80px !important; - } - - .mt-xs-10, -.my-xs-10 { - margin-top: 80px !important; - } - - .mr-xs-10, -.mx-xs-10 { - margin-right: 80px !important; - } - - .mb-xs-10, -.my-xs-10 { - margin-bottom: 80px !important; - } - - .ml-xs-10, -.mx-xs-10 { - margin-left: 80px !important; - } - - .m-xs-12 { - margin: 96px !important; - } - - .mt-xs-12, -.my-xs-12 { - margin-top: 96px !important; - } - - .mr-xs-12, -.mx-xs-12 { - margin-right: 96px !important; - } - - .mb-xs-12, -.my-xs-12 { - margin-bottom: 96px !important; - } - - .ml-xs-12, -.mx-xs-12 { - margin-left: 96px !important; - } - - .m-xs-15 { - margin: 120px !important; - } - - .mt-xs-15, -.my-xs-15 { - margin-top: 120px !important; - } - - .mr-xs-15, -.mx-xs-15 { - margin-right: 120px !important; - } - - .mb-xs-15, -.my-xs-15 { - margin-bottom: 120px !important; - } - - .ml-xs-15, -.mx-xs-15 { - margin-left: 120px !important; - } - - .p-xs-0 { - padding: 0 !important; - } - - .pt-xs-0, -.py-xs-0 { - padding-top: 0 !important; - } - - .pr-xs-0, -.px-xs-0 { - padding-right: 0 !important; - } - - .pb-xs-0, -.py-xs-0 { - padding-bottom: 0 !important; - } - - .pl-xs-0, -.px-xs-0 { - padding-left: 0 !important; - } - - .p-xs-1 { - padding: 8px !important; - } - - .pt-xs-1, -.py-xs-1 { - padding-top: 8px !important; - } - - .pr-xs-1, -.px-xs-1 { - padding-right: 8px !important; - } - - .pb-xs-1, -.py-xs-1 { - padding-bottom: 8px !important; - } - - .pl-xs-1, -.px-xs-1 { - padding-left: 8px !important; - } - - .p-xs-2 { - padding: 16px !important; - } - - .pt-xs-2, -.py-xs-2 { - padding-top: 16px !important; - } - - .pr-xs-2, -.px-xs-2 { - padding-right: 16px !important; - } - - .pb-xs-2, -.py-xs-2 { - padding-bottom: 16px !important; - } - - .pl-xs-2, -.px-xs-2 { - padding-left: 16px !important; - } - - .p-xs-3 { - padding: 24px !important; - } - - .pt-xs-3, -.py-xs-3 { - padding-top: 24px !important; - } - - .pr-xs-3, -.px-xs-3 { - padding-right: 24px !important; - } - - .pb-xs-3, -.py-xs-3 { - padding-bottom: 24px !important; - } - - .pl-xs-3, -.px-xs-3 { - padding-left: 24px !important; - } - - .p-xs-4 { - padding: 32px !important; - } - - .pt-xs-4, -.py-xs-4 { - padding-top: 32px !important; - } - - .pr-xs-4, -.px-xs-4 { - padding-right: 32px !important; - } - - .pb-xs-4, -.py-xs-4 { - padding-bottom: 32px !important; - } - - .pl-xs-4, -.px-xs-4 { - padding-left: 32px !important; - } - - .p-xs-5 { - padding: 40px !important; - } - - .pt-xs-5, -.py-xs-5 { - padding-top: 40px !important; - } - - .pr-xs-5, -.px-xs-5 { - padding-right: 40px !important; - } - - .pb-xs-5, -.py-xs-5 { - padding-bottom: 40px !important; - } - - .pl-xs-5, -.px-xs-5 { - padding-left: 40px !important; - } - - .p-xs-6 { - padding: 48px !important; - } - - .pt-xs-6, -.py-xs-6 { - padding-top: 48px !important; - } - - .pr-xs-6, -.px-xs-6 { - padding-right: 48px !important; - } - - .pb-xs-6, -.py-xs-6 { - padding-bottom: 48px !important; - } - - .pl-xs-6, -.px-xs-6 { - padding-left: 48px !important; - } - - .p-xs-7 { - padding: 56px !important; - } - - .pt-xs-7, -.py-xs-7 { - padding-top: 56px !important; - } - - .pr-xs-7, -.px-xs-7 { - padding-right: 56px !important; - } - - .pb-xs-7, -.py-xs-7 { - padding-bottom: 56px !important; - } - - .pl-xs-7, -.px-xs-7 { - padding-left: 56px !important; - } - - .p-xs-8 { - padding: 64px !important; - } - - .pt-xs-8, -.py-xs-8 { - padding-top: 64px !important; - } - - .pr-xs-8, -.px-xs-8 { - padding-right: 64px !important; - } - - .pb-xs-8, -.py-xs-8 { - padding-bottom: 64px !important; - } - - .pl-xs-8, -.px-xs-8 { - padding-left: 64px !important; - } - - .p-xs-9 { - padding: 72px !important; - } - - .pt-xs-9, -.py-xs-9 { - padding-top: 72px !important; - } - - .pr-xs-9, -.px-xs-9 { - padding-right: 72px !important; - } - - .pb-xs-9, -.py-xs-9 { - padding-bottom: 72px !important; - } - - .pl-xs-9, -.px-xs-9 { - padding-left: 72px !important; - } - - .p-xs-10 { - padding: 80px !important; - } - - .pt-xs-10, -.py-xs-10 { - padding-top: 80px !important; - } - - .pr-xs-10, -.px-xs-10 { - padding-right: 80px !important; - } - - .pb-xs-10, -.py-xs-10 { - padding-bottom: 80px !important; - } - - .pl-xs-10, -.px-xs-10 { - padding-left: 80px !important; - } - - .p-xs-12 { - padding: 96px !important; - } - - .pt-xs-12, -.py-xs-12 { - padding-top: 96px !important; - } - - .pr-xs-12, -.px-xs-12 { - padding-right: 96px !important; - } - - .pb-xs-12, -.py-xs-12 { - padding-bottom: 96px !important; - } - - .pl-xs-12, -.px-xs-12 { - padding-left: 96px !important; - } - - .p-xs-15 { - padding: 120px !important; - } - - .pt-xs-15, -.py-xs-15 { - padding-top: 120px !important; - } - - .pr-xs-15, -.px-xs-15 { - padding-right: 120px !important; - } - - .pb-xs-15, -.py-xs-15 { - padding-bottom: 120px !important; - } - - .pl-xs-15, -.px-xs-15 { - padding-left: 120px !important; - } - - .m-xs-n1 { - margin: -8px !important; - } - - .mt-xs-n1, -.my-xs-n1 { - margin-top: -8px !important; - } - - .mr-xs-n1, -.mx-xs-n1 { - margin-right: -8px !important; - } - - .mb-xs-n1, -.my-xs-n1 { - margin-bottom: -8px !important; - } - - .ml-xs-n1, -.mx-xs-n1 { - margin-left: -8px !important; - } - - .m-xs-n2 { - margin: -16px !important; - } - - .mt-xs-n2, -.my-xs-n2 { - margin-top: -16px !important; - } - - .mr-xs-n2, -.mx-xs-n2 { - margin-right: -16px !important; - } - - .mb-xs-n2, -.my-xs-n2 { - margin-bottom: -16px !important; - } - - .ml-xs-n2, -.mx-xs-n2 { - margin-left: -16px !important; - } - - .m-xs-n3 { - margin: -24px !important; - } - - .mt-xs-n3, -.my-xs-n3 { - margin-top: -24px !important; - } - - .mr-xs-n3, -.mx-xs-n3 { - margin-right: -24px !important; - } - - .mb-xs-n3, -.my-xs-n3 { - margin-bottom: -24px !important; - } - - .ml-xs-n3, -.mx-xs-n3 { - margin-left: -24px !important; - } - - .m-xs-n4 { - margin: -32px !important; - } - - .mt-xs-n4, -.my-xs-n4 { - margin-top: -32px !important; - } - - .mr-xs-n4, -.mx-xs-n4 { - margin-right: -32px !important; - } - - .mb-xs-n4, -.my-xs-n4 { - margin-bottom: -32px !important; - } - - .ml-xs-n4, -.mx-xs-n4 { - margin-left: -32px !important; - } - - .m-xs-n5 { - margin: -40px !important; - } - - .mt-xs-n5, -.my-xs-n5 { - margin-top: -40px !important; - } - - .mr-xs-n5, -.mx-xs-n5 { - margin-right: -40px !important; - } - - .mb-xs-n5, -.my-xs-n5 { - margin-bottom: -40px !important; - } - - .ml-xs-n5, -.mx-xs-n5 { - margin-left: -40px !important; - } - - .m-xs-n6 { - margin: -48px !important; - } - - .mt-xs-n6, -.my-xs-n6 { - margin-top: -48px !important; - } - - .mr-xs-n6, -.mx-xs-n6 { - margin-right: -48px !important; - } - - .mb-xs-n6, -.my-xs-n6 { - margin-bottom: -48px !important; - } - - .ml-xs-n6, -.mx-xs-n6 { - margin-left: -48px !important; - } - - .m-xs-n7 { - margin: -56px !important; - } - - .mt-xs-n7, -.my-xs-n7 { - margin-top: -56px !important; - } - - .mr-xs-n7, -.mx-xs-n7 { - margin-right: -56px !important; - } - - .mb-xs-n7, -.my-xs-n7 { - margin-bottom: -56px !important; - } - - .ml-xs-n7, -.mx-xs-n7 { - margin-left: -56px !important; - } - - .m-xs-n8 { - margin: -64px !important; - } - - .mt-xs-n8, -.my-xs-n8 { - margin-top: -64px !important; - } - - .mr-xs-n8, -.mx-xs-n8 { - margin-right: -64px !important; - } - - .mb-xs-n8, -.my-xs-n8 { - margin-bottom: -64px !important; - } - - .ml-xs-n8, -.mx-xs-n8 { - margin-left: -64px !important; - } - - .m-xs-n9 { - margin: -72px !important; - } - - .mt-xs-n9, -.my-xs-n9 { - margin-top: -72px !important; - } - - .mr-xs-n9, -.mx-xs-n9 { - margin-right: -72px !important; - } - - .mb-xs-n9, -.my-xs-n9 { - margin-bottom: -72px !important; - } - - .ml-xs-n9, -.mx-xs-n9 { - margin-left: -72px !important; - } - - .m-xs-n10 { - margin: -80px !important; - } - - .mt-xs-n10, -.my-xs-n10 { - margin-top: -80px !important; - } - - .mr-xs-n10, -.mx-xs-n10 { - margin-right: -80px !important; - } - - .mb-xs-n10, -.my-xs-n10 { - margin-bottom: -80px !important; - } - - .ml-xs-n10, -.mx-xs-n10 { - margin-left: -80px !important; - } - - .m-xs-n12 { - margin: -96px !important; - } - - .mt-xs-n12, -.my-xs-n12 { - margin-top: -96px !important; - } - - .mr-xs-n12, -.mx-xs-n12 { - margin-right: -96px !important; - } - - .mb-xs-n12, -.my-xs-n12 { - margin-bottom: -96px !important; - } - - .ml-xs-n12, -.mx-xs-n12 { - margin-left: -96px !important; - } - - .m-xs-n15 { - margin: -120px !important; - } - - .mt-xs-n15, -.my-xs-n15 { - margin-top: -120px !important; - } - - .mr-xs-n15, -.mx-xs-n15 { - margin-right: -120px !important; - } - - .mb-xs-n15, -.my-xs-n15 { - margin-bottom: -120px !important; - } - - .ml-xs-n15, -.mx-xs-n15 { - margin-left: -120px !important; - } - - .m-xs-auto { - margin: auto !important; - } - - .mt-xs-auto, -.my-xs-auto { - margin-top: auto !important; - } - - .mr-xs-auto, -.mx-xs-auto { - margin-right: auto !important; - } - - .mb-xs-auto, -.my-xs-auto { - margin-bottom: auto !important; - } - - .ml-xs-auto, -.mx-xs-auto { - margin-left: auto !important; - } -} -@media (min-width: 616px) { - .m-sm-0 { - margin: 0 !important; - } - - .mt-sm-0, -.my-sm-0 { - margin-top: 0 !important; - } - - .mr-sm-0, -.mx-sm-0 { - margin-right: 0 !important; - } - - .mb-sm-0, -.my-sm-0 { - margin-bottom: 0 !important; - } - - .ml-sm-0, -.mx-sm-0 { - margin-left: 0 !important; - } - - .m-sm-1 { - margin: 8px !important; - } - - .mt-sm-1, -.my-sm-1 { - margin-top: 8px !important; - } - - .mr-sm-1, -.mx-sm-1 { - margin-right: 8px !important; - } - - .mb-sm-1, -.my-sm-1 { - margin-bottom: 8px !important; - } - - .ml-sm-1, -.mx-sm-1 { - margin-left: 8px !important; - } - - .m-sm-2 { - margin: 16px !important; - } - - .mt-sm-2, -.my-sm-2 { - margin-top: 16px !important; - } - - .mr-sm-2, -.mx-sm-2 { - margin-right: 16px !important; - } - - .mb-sm-2, -.my-sm-2 { - margin-bottom: 16px !important; - } - - .ml-sm-2, -.mx-sm-2 { - margin-left: 16px !important; - } - - .m-sm-3 { - margin: 24px !important; - } - - .mt-sm-3, -.my-sm-3 { - margin-top: 24px !important; - } - - .mr-sm-3, -.mx-sm-3 { - margin-right: 24px !important; - } - - .mb-sm-3, -.my-sm-3 { - margin-bottom: 24px !important; - } - - .ml-sm-3, -.mx-sm-3 { - margin-left: 24px !important; - } - - .m-sm-4 { - margin: 32px !important; - } - - .mt-sm-4, -.my-sm-4 { - margin-top: 32px !important; - } - - .mr-sm-4, -.mx-sm-4 { - margin-right: 32px !important; - } - - .mb-sm-4, -.my-sm-4 { - margin-bottom: 32px !important; - } - - .ml-sm-4, -.mx-sm-4 { - margin-left: 32px !important; - } - - .m-sm-5 { - margin: 40px !important; - } - - .mt-sm-5, -.my-sm-5 { - margin-top: 40px !important; - } - - .mr-sm-5, -.mx-sm-5 { - margin-right: 40px !important; - } - - .mb-sm-5, -.my-sm-5 { - margin-bottom: 40px !important; - } - - .ml-sm-5, -.mx-sm-5 { - margin-left: 40px !important; - } - - .m-sm-6 { - margin: 48px !important; - } - - .mt-sm-6, -.my-sm-6 { - margin-top: 48px !important; - } - - .mr-sm-6, -.mx-sm-6 { - margin-right: 48px !important; - } - - .mb-sm-6, -.my-sm-6 { - margin-bottom: 48px !important; - } - - .ml-sm-6, -.mx-sm-6 { - margin-left: 48px !important; - } - - .m-sm-7 { - margin: 56px !important; - } - - .mt-sm-7, -.my-sm-7 { - margin-top: 56px !important; - } - - .mr-sm-7, -.mx-sm-7 { - margin-right: 56px !important; - } - - .mb-sm-7, -.my-sm-7 { - margin-bottom: 56px !important; - } - - .ml-sm-7, -.mx-sm-7 { - margin-left: 56px !important; - } - - .m-sm-8 { - margin: 64px !important; - } - - .mt-sm-8, -.my-sm-8 { - margin-top: 64px !important; - } - - .mr-sm-8, -.mx-sm-8 { - margin-right: 64px !important; - } - - .mb-sm-8, -.my-sm-8 { - margin-bottom: 64px !important; - } - - .ml-sm-8, -.mx-sm-8 { - margin-left: 64px !important; - } - - .m-sm-9 { - margin: 72px !important; - } - - .mt-sm-9, -.my-sm-9 { - margin-top: 72px !important; - } - - .mr-sm-9, -.mx-sm-9 { - margin-right: 72px !important; - } - - .mb-sm-9, -.my-sm-9 { - margin-bottom: 72px !important; - } - - .ml-sm-9, -.mx-sm-9 { - margin-left: 72px !important; - } - - .m-sm-10 { - margin: 80px !important; - } - - .mt-sm-10, -.my-sm-10 { - margin-top: 80px !important; - } - - .mr-sm-10, -.mx-sm-10 { - margin-right: 80px !important; - } - - .mb-sm-10, -.my-sm-10 { - margin-bottom: 80px !important; - } - - .ml-sm-10, -.mx-sm-10 { - margin-left: 80px !important; - } - - .m-sm-12 { - margin: 96px !important; - } - - .mt-sm-12, -.my-sm-12 { - margin-top: 96px !important; - } - - .mr-sm-12, -.mx-sm-12 { - margin-right: 96px !important; - } - - .mb-sm-12, -.my-sm-12 { - margin-bottom: 96px !important; - } - - .ml-sm-12, -.mx-sm-12 { - margin-left: 96px !important; - } - - .m-sm-15 { - margin: 120px !important; - } - - .mt-sm-15, -.my-sm-15 { - margin-top: 120px !important; - } - - .mr-sm-15, -.mx-sm-15 { - margin-right: 120px !important; - } - - .mb-sm-15, -.my-sm-15 { - margin-bottom: 120px !important; - } - - .ml-sm-15, -.mx-sm-15 { - margin-left: 120px !important; - } - - .p-sm-0 { - padding: 0 !important; - } - - .pt-sm-0, -.py-sm-0 { - padding-top: 0 !important; - } - - .pr-sm-0, -.px-sm-0 { - padding-right: 0 !important; - } - - .pb-sm-0, -.py-sm-0 { - padding-bottom: 0 !important; - } - - .pl-sm-0, -.px-sm-0 { - padding-left: 0 !important; - } - - .p-sm-1 { - padding: 8px !important; - } - - .pt-sm-1, -.py-sm-1 { - padding-top: 8px !important; - } - - .pr-sm-1, -.px-sm-1 { - padding-right: 8px !important; - } - - .pb-sm-1, -.py-sm-1 { - padding-bottom: 8px !important; - } - - .pl-sm-1, -.px-sm-1 { - padding-left: 8px !important; - } - - .p-sm-2 { - padding: 16px !important; - } - - .pt-sm-2, -.py-sm-2 { - padding-top: 16px !important; - } - - .pr-sm-2, -.px-sm-2 { - padding-right: 16px !important; - } - - .pb-sm-2, -.py-sm-2 { - padding-bottom: 16px !important; - } - - .pl-sm-2, -.px-sm-2 { - padding-left: 16px !important; - } - - .p-sm-3 { - padding: 24px !important; - } - - .pt-sm-3, -.py-sm-3 { - padding-top: 24px !important; - } - - .pr-sm-3, -.px-sm-3 { - padding-right: 24px !important; - } - - .pb-sm-3, -.py-sm-3 { - padding-bottom: 24px !important; - } - - .pl-sm-3, -.px-sm-3 { - padding-left: 24px !important; - } - - .p-sm-4 { - padding: 32px !important; - } - - .pt-sm-4, -.py-sm-4 { - padding-top: 32px !important; - } - - .pr-sm-4, -.px-sm-4 { - padding-right: 32px !important; - } - - .pb-sm-4, -.py-sm-4 { - padding-bottom: 32px !important; - } - - .pl-sm-4, -.px-sm-4 { - padding-left: 32px !important; - } - - .p-sm-5 { - padding: 40px !important; - } - - .pt-sm-5, -.py-sm-5 { - padding-top: 40px !important; - } - - .pr-sm-5, -.px-sm-5 { - padding-right: 40px !important; - } - - .pb-sm-5, -.py-sm-5 { - padding-bottom: 40px !important; - } - - .pl-sm-5, -.px-sm-5 { - padding-left: 40px !important; - } - - .p-sm-6 { - padding: 48px !important; - } - - .pt-sm-6, -.py-sm-6 { - padding-top: 48px !important; - } - - .pr-sm-6, -.px-sm-6 { - padding-right: 48px !important; - } - - .pb-sm-6, -.py-sm-6 { - padding-bottom: 48px !important; - } - - .pl-sm-6, -.px-sm-6 { - padding-left: 48px !important; - } - - .p-sm-7 { - padding: 56px !important; - } - - .pt-sm-7, -.py-sm-7 { - padding-top: 56px !important; - } - - .pr-sm-7, -.px-sm-7 { - padding-right: 56px !important; - } - - .pb-sm-7, -.py-sm-7 { - padding-bottom: 56px !important; - } - - .pl-sm-7, -.px-sm-7 { - padding-left: 56px !important; - } - - .p-sm-8 { - padding: 64px !important; - } - - .pt-sm-8, -.py-sm-8 { - padding-top: 64px !important; - } - - .pr-sm-8, -.px-sm-8 { - padding-right: 64px !important; - } - - .pb-sm-8, -.py-sm-8 { - padding-bottom: 64px !important; - } - - .pl-sm-8, -.px-sm-8 { - padding-left: 64px !important; - } - - .p-sm-9 { - padding: 72px !important; - } - - .pt-sm-9, -.py-sm-9 { - padding-top: 72px !important; - } - - .pr-sm-9, -.px-sm-9 { - padding-right: 72px !important; - } - - .pb-sm-9, -.py-sm-9 { - padding-bottom: 72px !important; - } - - .pl-sm-9, -.px-sm-9 { - padding-left: 72px !important; - } - - .p-sm-10 { - padding: 80px !important; - } - - .pt-sm-10, -.py-sm-10 { - padding-top: 80px !important; - } - - .pr-sm-10, -.px-sm-10 { - padding-right: 80px !important; - } - - .pb-sm-10, -.py-sm-10 { - padding-bottom: 80px !important; - } - - .pl-sm-10, -.px-sm-10 { - padding-left: 80px !important; - } - - .p-sm-12 { - padding: 96px !important; - } - - .pt-sm-12, -.py-sm-12 { - padding-top: 96px !important; - } - - .pr-sm-12, -.px-sm-12 { - padding-right: 96px !important; - } - - .pb-sm-12, -.py-sm-12 { - padding-bottom: 96px !important; - } - - .pl-sm-12, -.px-sm-12 { - padding-left: 96px !important; - } - - .p-sm-15 { - padding: 120px !important; - } - - .pt-sm-15, -.py-sm-15 { - padding-top: 120px !important; - } - - .pr-sm-15, -.px-sm-15 { - padding-right: 120px !important; - } - - .pb-sm-15, -.py-sm-15 { - padding-bottom: 120px !important; - } - - .pl-sm-15, -.px-sm-15 { - padding-left: 120px !important; - } - - .m-sm-n1 { - margin: -8px !important; - } - - .mt-sm-n1, -.my-sm-n1 { - margin-top: -8px !important; - } - - .mr-sm-n1, -.mx-sm-n1 { - margin-right: -8px !important; - } - - .mb-sm-n1, -.my-sm-n1 { - margin-bottom: -8px !important; - } - - .ml-sm-n1, -.mx-sm-n1 { - margin-left: -8px !important; - } - - .m-sm-n2 { - margin: -16px !important; - } - - .mt-sm-n2, -.my-sm-n2 { - margin-top: -16px !important; - } - - .mr-sm-n2, -.mx-sm-n2 { - margin-right: -16px !important; - } - - .mb-sm-n2, -.my-sm-n2 { - margin-bottom: -16px !important; - } - - .ml-sm-n2, -.mx-sm-n2 { - margin-left: -16px !important; - } - - .m-sm-n3 { - margin: -24px !important; - } - - .mt-sm-n3, -.my-sm-n3 { - margin-top: -24px !important; - } - - .mr-sm-n3, -.mx-sm-n3 { - margin-right: -24px !important; - } - - .mb-sm-n3, -.my-sm-n3 { - margin-bottom: -24px !important; - } - - .ml-sm-n3, -.mx-sm-n3 { - margin-left: -24px !important; - } - - .m-sm-n4 { - margin: -32px !important; - } - - .mt-sm-n4, -.my-sm-n4 { - margin-top: -32px !important; - } - - .mr-sm-n4, -.mx-sm-n4 { - margin-right: -32px !important; - } - - .mb-sm-n4, -.my-sm-n4 { - margin-bottom: -32px !important; - } - - .ml-sm-n4, -.mx-sm-n4 { - margin-left: -32px !important; - } - - .m-sm-n5 { - margin: -40px !important; - } - - .mt-sm-n5, -.my-sm-n5 { - margin-top: -40px !important; - } - - .mr-sm-n5, -.mx-sm-n5 { - margin-right: -40px !important; - } - - .mb-sm-n5, -.my-sm-n5 { - margin-bottom: -40px !important; - } - - .ml-sm-n5, -.mx-sm-n5 { - margin-left: -40px !important; - } - - .m-sm-n6 { - margin: -48px !important; - } - - .mt-sm-n6, -.my-sm-n6 { - margin-top: -48px !important; - } - - .mr-sm-n6, -.mx-sm-n6 { - margin-right: -48px !important; - } - - .mb-sm-n6, -.my-sm-n6 { - margin-bottom: -48px !important; - } - - .ml-sm-n6, -.mx-sm-n6 { - margin-left: -48px !important; - } - - .m-sm-n7 { - margin: -56px !important; - } - - .mt-sm-n7, -.my-sm-n7 { - margin-top: -56px !important; - } - - .mr-sm-n7, -.mx-sm-n7 { - margin-right: -56px !important; - } - - .mb-sm-n7, -.my-sm-n7 { - margin-bottom: -56px !important; - } - - .ml-sm-n7, -.mx-sm-n7 { - margin-left: -56px !important; - } - - .m-sm-n8 { - margin: -64px !important; - } - - .mt-sm-n8, -.my-sm-n8 { - margin-top: -64px !important; - } - - .mr-sm-n8, -.mx-sm-n8 { - margin-right: -64px !important; - } - - .mb-sm-n8, -.my-sm-n8 { - margin-bottom: -64px !important; - } - - .ml-sm-n8, -.mx-sm-n8 { - margin-left: -64px !important; - } - - .m-sm-n9 { - margin: -72px !important; - } - - .mt-sm-n9, -.my-sm-n9 { - margin-top: -72px !important; - } - - .mr-sm-n9, -.mx-sm-n9 { - margin-right: -72px !important; - } - - .mb-sm-n9, -.my-sm-n9 { - margin-bottom: -72px !important; - } - - .ml-sm-n9, -.mx-sm-n9 { - margin-left: -72px !important; - } - - .m-sm-n10 { - margin: -80px !important; - } - - .mt-sm-n10, -.my-sm-n10 { - margin-top: -80px !important; - } - - .mr-sm-n10, -.mx-sm-n10 { - margin-right: -80px !important; - } - - .mb-sm-n10, -.my-sm-n10 { - margin-bottom: -80px !important; - } - - .ml-sm-n10, -.mx-sm-n10 { - margin-left: -80px !important; - } - - .m-sm-n12 { - margin: -96px !important; - } - - .mt-sm-n12, -.my-sm-n12 { - margin-top: -96px !important; - } - - .mr-sm-n12, -.mx-sm-n12 { - margin-right: -96px !important; - } - - .mb-sm-n12, -.my-sm-n12 { - margin-bottom: -96px !important; - } - - .ml-sm-n12, -.mx-sm-n12 { - margin-left: -96px !important; - } - - .m-sm-n15 { - margin: -120px !important; - } - - .mt-sm-n15, -.my-sm-n15 { - margin-top: -120px !important; - } - - .mr-sm-n15, -.mx-sm-n15 { - margin-right: -120px !important; - } - - .mb-sm-n15, -.my-sm-n15 { - margin-bottom: -120px !important; - } - - .ml-sm-n15, -.mx-sm-n15 { - margin-left: -120px !important; - } - - .m-sm-auto { - margin: auto !important; - } - - .mt-sm-auto, -.my-sm-auto { - margin-top: auto !important; - } - - .mr-sm-auto, -.mx-sm-auto { - margin-right: auto !important; - } - - .mb-sm-auto, -.my-sm-auto { - margin-bottom: auto !important; - } - - .ml-sm-auto, -.mx-sm-auto { - margin-left: auto !important; - } -} -@media (min-width: 768px) { - .m-md-0 { - margin: 0 !important; - } - - .mt-md-0, -.my-md-0 { - margin-top: 0 !important; - } - - .mr-md-0, -.mx-md-0 { - margin-right: 0 !important; - } - - .mb-md-0, -.my-md-0 { - margin-bottom: 0 !important; - } - - .ml-md-0, -.mx-md-0 { - margin-left: 0 !important; - } - - .m-md-1 { - margin: 8px !important; - } - - .mt-md-1, -.my-md-1 { - margin-top: 8px !important; - } - - .mr-md-1, -.mx-md-1 { - margin-right: 8px !important; - } - - .mb-md-1, -.my-md-1 { - margin-bottom: 8px !important; - } - - .ml-md-1, -.mx-md-1 { - margin-left: 8px !important; - } - - .m-md-2 { - margin: 16px !important; - } - - .mt-md-2, -.my-md-2 { - margin-top: 16px !important; - } - - .mr-md-2, -.mx-md-2 { - margin-right: 16px !important; - } - - .mb-md-2, -.my-md-2 { - margin-bottom: 16px !important; - } - - .ml-md-2, -.mx-md-2 { - margin-left: 16px !important; - } - - .m-md-3 { - margin: 24px !important; - } - - .mt-md-3, -.my-md-3 { - margin-top: 24px !important; - } - - .mr-md-3, -.mx-md-3 { - margin-right: 24px !important; - } - - .mb-md-3, -.my-md-3 { - margin-bottom: 24px !important; - } - - .ml-md-3, -.mx-md-3 { - margin-left: 24px !important; - } - - .m-md-4 { - margin: 32px !important; - } - - .mt-md-4, -.my-md-4 { - margin-top: 32px !important; - } - - .mr-md-4, -.mx-md-4 { - margin-right: 32px !important; - } - - .mb-md-4, -.my-md-4 { - margin-bottom: 32px !important; - } - - .ml-md-4, -.mx-md-4 { - margin-left: 32px !important; - } - - .m-md-5 { - margin: 40px !important; - } - - .mt-md-5, -.my-md-5 { - margin-top: 40px !important; - } - - .mr-md-5, -.mx-md-5 { - margin-right: 40px !important; - } - - .mb-md-5, -.my-md-5 { - margin-bottom: 40px !important; - } - - .ml-md-5, -.mx-md-5 { - margin-left: 40px !important; - } - - .m-md-6 { - margin: 48px !important; - } - - .mt-md-6, -.my-md-6 { - margin-top: 48px !important; - } - - .mr-md-6, -.mx-md-6 { - margin-right: 48px !important; - } - - .mb-md-6, -.my-md-6 { - margin-bottom: 48px !important; - } - - .ml-md-6, -.mx-md-6 { - margin-left: 48px !important; - } - - .m-md-7 { - margin: 56px !important; - } - - .mt-md-7, -.my-md-7 { - margin-top: 56px !important; - } - - .mr-md-7, -.mx-md-7 { - margin-right: 56px !important; - } - - .mb-md-7, -.my-md-7 { - margin-bottom: 56px !important; - } - - .ml-md-7, -.mx-md-7 { - margin-left: 56px !important; - } - - .m-md-8 { - margin: 64px !important; - } - - .mt-md-8, -.my-md-8 { - margin-top: 64px !important; - } - - .mr-md-8, -.mx-md-8 { - margin-right: 64px !important; - } - - .mb-md-8, -.my-md-8 { - margin-bottom: 64px !important; - } - - .ml-md-8, -.mx-md-8 { - margin-left: 64px !important; - } - - .m-md-9 { - margin: 72px !important; - } - - .mt-md-9, -.my-md-9 { - margin-top: 72px !important; - } - - .mr-md-9, -.mx-md-9 { - margin-right: 72px !important; - } - - .mb-md-9, -.my-md-9 { - margin-bottom: 72px !important; - } - - .ml-md-9, -.mx-md-9 { - margin-left: 72px !important; - } - - .m-md-10 { - margin: 80px !important; - } - - .mt-md-10, -.my-md-10 { - margin-top: 80px !important; - } - - .mr-md-10, -.mx-md-10 { - margin-right: 80px !important; - } - - .mb-md-10, -.my-md-10 { - margin-bottom: 80px !important; - } - - .ml-md-10, -.mx-md-10 { - margin-left: 80px !important; - } - - .m-md-12 { - margin: 96px !important; - } - - .mt-md-12, -.my-md-12 { - margin-top: 96px !important; - } - - .mr-md-12, -.mx-md-12 { - margin-right: 96px !important; - } - - .mb-md-12, -.my-md-12 { - margin-bottom: 96px !important; - } - - .ml-md-12, -.mx-md-12 { - margin-left: 96px !important; - } - - .m-md-15 { - margin: 120px !important; - } - - .mt-md-15, -.my-md-15 { - margin-top: 120px !important; - } - - .mr-md-15, -.mx-md-15 { - margin-right: 120px !important; - } - - .mb-md-15, -.my-md-15 { - margin-bottom: 120px !important; - } - - .ml-md-15, -.mx-md-15 { - margin-left: 120px !important; - } - - .p-md-0 { - padding: 0 !important; - } - - .pt-md-0, -.py-md-0 { - padding-top: 0 !important; - } - - .pr-md-0, -.px-md-0 { - padding-right: 0 !important; - } - - .pb-md-0, -.py-md-0 { - padding-bottom: 0 !important; - } - - .pl-md-0, -.px-md-0 { - padding-left: 0 !important; - } - - .p-md-1 { - padding: 8px !important; - } - - .pt-md-1, -.py-md-1 { - padding-top: 8px !important; - } - - .pr-md-1, -.px-md-1 { - padding-right: 8px !important; - } - - .pb-md-1, -.py-md-1 { - padding-bottom: 8px !important; - } - - .pl-md-1, -.px-md-1 { - padding-left: 8px !important; - } - - .p-md-2 { - padding: 16px !important; - } - - .pt-md-2, -.py-md-2 { - padding-top: 16px !important; - } - - .pr-md-2, -.px-md-2 { - padding-right: 16px !important; - } - - .pb-md-2, -.py-md-2 { - padding-bottom: 16px !important; - } - - .pl-md-2, -.px-md-2 { - padding-left: 16px !important; - } - - .p-md-3 { - padding: 24px !important; - } - - .pt-md-3, -.py-md-3 { - padding-top: 24px !important; - } - - .pr-md-3, -.px-md-3 { - padding-right: 24px !important; - } - - .pb-md-3, -.py-md-3 { - padding-bottom: 24px !important; - } - - .pl-md-3, -.px-md-3 { - padding-left: 24px !important; - } - - .p-md-4 { - padding: 32px !important; - } - - .pt-md-4, -.py-md-4 { - padding-top: 32px !important; - } - - .pr-md-4, -.px-md-4 { - padding-right: 32px !important; - } - - .pb-md-4, -.py-md-4 { - padding-bottom: 32px !important; - } - - .pl-md-4, -.px-md-4 { - padding-left: 32px !important; - } - - .p-md-5 { - padding: 40px !important; - } - - .pt-md-5, -.py-md-5 { - padding-top: 40px !important; - } - - .pr-md-5, -.px-md-5 { - padding-right: 40px !important; - } - - .pb-md-5, -.py-md-5 { - padding-bottom: 40px !important; - } - - .pl-md-5, -.px-md-5 { - padding-left: 40px !important; - } - - .p-md-6 { - padding: 48px !important; - } - - .pt-md-6, -.py-md-6 { - padding-top: 48px !important; - } - - .pr-md-6, -.px-md-6 { - padding-right: 48px !important; - } - - .pb-md-6, -.py-md-6 { - padding-bottom: 48px !important; - } - - .pl-md-6, -.px-md-6 { - padding-left: 48px !important; - } - - .p-md-7 { - padding: 56px !important; - } - - .pt-md-7, -.py-md-7 { - padding-top: 56px !important; - } - - .pr-md-7, -.px-md-7 { - padding-right: 56px !important; - } - - .pb-md-7, -.py-md-7 { - padding-bottom: 56px !important; - } - - .pl-md-7, -.px-md-7 { - padding-left: 56px !important; - } - - .p-md-8 { - padding: 64px !important; - } - - .pt-md-8, -.py-md-8 { - padding-top: 64px !important; - } - - .pr-md-8, -.px-md-8 { - padding-right: 64px !important; - } - - .pb-md-8, -.py-md-8 { - padding-bottom: 64px !important; - } - - .pl-md-8, -.px-md-8 { - padding-left: 64px !important; - } - - .p-md-9 { - padding: 72px !important; - } - - .pt-md-9, -.py-md-9 { - padding-top: 72px !important; - } - - .pr-md-9, -.px-md-9 { - padding-right: 72px !important; - } - - .pb-md-9, -.py-md-9 { - padding-bottom: 72px !important; - } - - .pl-md-9, -.px-md-9 { - padding-left: 72px !important; - } - - .p-md-10 { - padding: 80px !important; - } - - .pt-md-10, -.py-md-10 { - padding-top: 80px !important; - } - - .pr-md-10, -.px-md-10 { - padding-right: 80px !important; - } - - .pb-md-10, -.py-md-10 { - padding-bottom: 80px !important; - } - - .pl-md-10, -.px-md-10 { - padding-left: 80px !important; - } - - .p-md-12 { - padding: 96px !important; - } - - .pt-md-12, -.py-md-12 { - padding-top: 96px !important; - } - - .pr-md-12, -.px-md-12 { - padding-right: 96px !important; - } - - .pb-md-12, -.py-md-12 { - padding-bottom: 96px !important; - } - - .pl-md-12, -.px-md-12 { - padding-left: 96px !important; - } - - .p-md-15 { - padding: 120px !important; - } - - .pt-md-15, -.py-md-15 { - padding-top: 120px !important; - } - - .pr-md-15, -.px-md-15 { - padding-right: 120px !important; - } - - .pb-md-15, -.py-md-15 { - padding-bottom: 120px !important; - } - - .pl-md-15, -.px-md-15 { - padding-left: 120px !important; - } - - .m-md-n1 { - margin: -8px !important; - } - - .mt-md-n1, -.my-md-n1 { - margin-top: -8px !important; - } - - .mr-md-n1, -.mx-md-n1 { - margin-right: -8px !important; - } - - .mb-md-n1, -.my-md-n1 { - margin-bottom: -8px !important; - } - - .ml-md-n1, -.mx-md-n1 { - margin-left: -8px !important; - } - - .m-md-n2 { - margin: -16px !important; - } - - .mt-md-n2, -.my-md-n2 { - margin-top: -16px !important; - } - - .mr-md-n2, -.mx-md-n2 { - margin-right: -16px !important; - } - - .mb-md-n2, -.my-md-n2 { - margin-bottom: -16px !important; - } - - .ml-md-n2, -.mx-md-n2 { - margin-left: -16px !important; - } - - .m-md-n3 { - margin: -24px !important; - } - - .mt-md-n3, -.my-md-n3 { - margin-top: -24px !important; - } - - .mr-md-n3, -.mx-md-n3 { - margin-right: -24px !important; - } - - .mb-md-n3, -.my-md-n3 { - margin-bottom: -24px !important; - } - - .ml-md-n3, -.mx-md-n3 { - margin-left: -24px !important; - } - - .m-md-n4 { - margin: -32px !important; - } - - .mt-md-n4, -.my-md-n4 { - margin-top: -32px !important; - } - - .mr-md-n4, -.mx-md-n4 { - margin-right: -32px !important; - } - - .mb-md-n4, -.my-md-n4 { - margin-bottom: -32px !important; - } - - .ml-md-n4, -.mx-md-n4 { - margin-left: -32px !important; - } - - .m-md-n5 { - margin: -40px !important; - } - - .mt-md-n5, -.my-md-n5 { - margin-top: -40px !important; - } - - .mr-md-n5, -.mx-md-n5 { - margin-right: -40px !important; - } - - .mb-md-n5, -.my-md-n5 { - margin-bottom: -40px !important; - } - - .ml-md-n5, -.mx-md-n5 { - margin-left: -40px !important; - } - - .m-md-n6 { - margin: -48px !important; - } - - .mt-md-n6, -.my-md-n6 { - margin-top: -48px !important; - } - - .mr-md-n6, -.mx-md-n6 { - margin-right: -48px !important; - } - - .mb-md-n6, -.my-md-n6 { - margin-bottom: -48px !important; - } - - .ml-md-n6, -.mx-md-n6 { - margin-left: -48px !important; - } - - .m-md-n7 { - margin: -56px !important; - } - - .mt-md-n7, -.my-md-n7 { - margin-top: -56px !important; - } - - .mr-md-n7, -.mx-md-n7 { - margin-right: -56px !important; - } - - .mb-md-n7, -.my-md-n7 { - margin-bottom: -56px !important; - } - - .ml-md-n7, -.mx-md-n7 { - margin-left: -56px !important; - } - - .m-md-n8 { - margin: -64px !important; - } - - .mt-md-n8, -.my-md-n8 { - margin-top: -64px !important; - } - - .mr-md-n8, -.mx-md-n8 { - margin-right: -64px !important; - } - - .mb-md-n8, -.my-md-n8 { - margin-bottom: -64px !important; - } - - .ml-md-n8, -.mx-md-n8 { - margin-left: -64px !important; - } - - .m-md-n9 { - margin: -72px !important; - } - - .mt-md-n9, -.my-md-n9 { - margin-top: -72px !important; - } - - .mr-md-n9, -.mx-md-n9 { - margin-right: -72px !important; - } - - .mb-md-n9, -.my-md-n9 { - margin-bottom: -72px !important; - } - - .ml-md-n9, -.mx-md-n9 { - margin-left: -72px !important; - } - - .m-md-n10 { - margin: -80px !important; - } - - .mt-md-n10, -.my-md-n10 { - margin-top: -80px !important; - } - - .mr-md-n10, -.mx-md-n10 { - margin-right: -80px !important; - } - - .mb-md-n10, -.my-md-n10 { - margin-bottom: -80px !important; - } - - .ml-md-n10, -.mx-md-n10 { - margin-left: -80px !important; - } - - .m-md-n12 { - margin: -96px !important; - } - - .mt-md-n12, -.my-md-n12 { - margin-top: -96px !important; - } - - .mr-md-n12, -.mx-md-n12 { - margin-right: -96px !important; - } - - .mb-md-n12, -.my-md-n12 { - margin-bottom: -96px !important; - } - - .ml-md-n12, -.mx-md-n12 { - margin-left: -96px !important; - } - - .m-md-n15 { - margin: -120px !important; - } - - .mt-md-n15, -.my-md-n15 { - margin-top: -120px !important; - } - - .mr-md-n15, -.mx-md-n15 { - margin-right: -120px !important; - } - - .mb-md-n15, -.my-md-n15 { - margin-bottom: -120px !important; - } - - .ml-md-n15, -.mx-md-n15 { - margin-left: -120px !important; - } - - .m-md-auto { - margin: auto !important; - } - - .mt-md-auto, -.my-md-auto { - margin-top: auto !important; - } - - .mr-md-auto, -.mx-md-auto { - margin-right: auto !important; - } - - .mb-md-auto, -.my-md-auto { - margin-bottom: auto !important; - } - - .ml-md-auto, -.mx-md-auto { - margin-left: auto !important; - } -} -@media (min-width: 980px) { - .m-lg-0 { - margin: 0 !important; - } - - .mt-lg-0, -.my-lg-0 { - margin-top: 0 !important; - } - - .mr-lg-0, -.mx-lg-0 { - margin-right: 0 !important; - } - - .mb-lg-0, -.my-lg-0 { - margin-bottom: 0 !important; - } - - .ml-lg-0, -.mx-lg-0 { - margin-left: 0 !important; - } - - .m-lg-1 { - margin: 8px !important; - } - - .mt-lg-1, -.my-lg-1 { - margin-top: 8px !important; - } - - .mr-lg-1, -.mx-lg-1 { - margin-right: 8px !important; - } - - .mb-lg-1, -.my-lg-1 { - margin-bottom: 8px !important; - } - - .ml-lg-1, -.mx-lg-1 { - margin-left: 8px !important; - } - - .m-lg-2 { - margin: 16px !important; - } - - .mt-lg-2, -.my-lg-2 { - margin-top: 16px !important; - } - - .mr-lg-2, -.mx-lg-2 { - margin-right: 16px !important; - } - - .mb-lg-2, -.my-lg-2 { - margin-bottom: 16px !important; - } - - .ml-lg-2, -.mx-lg-2 { - margin-left: 16px !important; - } - - .m-lg-3 { - margin: 24px !important; - } - - .mt-lg-3, -.my-lg-3 { - margin-top: 24px !important; - } - - .mr-lg-3, -.mx-lg-3 { - margin-right: 24px !important; - } - - .mb-lg-3, -.my-lg-3 { - margin-bottom: 24px !important; - } - - .ml-lg-3, -.mx-lg-3 { - margin-left: 24px !important; - } - - .m-lg-4 { - margin: 32px !important; - } - - .mt-lg-4, -.my-lg-4 { - margin-top: 32px !important; - } - - .mr-lg-4, -.mx-lg-4 { - margin-right: 32px !important; - } - - .mb-lg-4, -.my-lg-4 { - margin-bottom: 32px !important; - } - - .ml-lg-4, -.mx-lg-4 { - margin-left: 32px !important; - } - - .m-lg-5 { - margin: 40px !important; - } - - .mt-lg-5, -.my-lg-5 { - margin-top: 40px !important; - } - - .mr-lg-5, -.mx-lg-5 { - margin-right: 40px !important; - } - - .mb-lg-5, -.my-lg-5 { - margin-bottom: 40px !important; - } - - .ml-lg-5, -.mx-lg-5 { - margin-left: 40px !important; - } - - .m-lg-6 { - margin: 48px !important; - } - - .mt-lg-6, -.my-lg-6 { - margin-top: 48px !important; - } - - .mr-lg-6, -.mx-lg-6 { - margin-right: 48px !important; - } - - .mb-lg-6, -.my-lg-6 { - margin-bottom: 48px !important; - } - - .ml-lg-6, -.mx-lg-6 { - margin-left: 48px !important; - } - - .m-lg-7 { - margin: 56px !important; - } - - .mt-lg-7, -.my-lg-7 { - margin-top: 56px !important; - } - - .mr-lg-7, -.mx-lg-7 { - margin-right: 56px !important; - } - - .mb-lg-7, -.my-lg-7 { - margin-bottom: 56px !important; - } - - .ml-lg-7, -.mx-lg-7 { - margin-left: 56px !important; - } - - .m-lg-8 { - margin: 64px !important; - } - - .mt-lg-8, -.my-lg-8 { - margin-top: 64px !important; - } - - .mr-lg-8, -.mx-lg-8 { - margin-right: 64px !important; - } - - .mb-lg-8, -.my-lg-8 { - margin-bottom: 64px !important; - } - - .ml-lg-8, -.mx-lg-8 { - margin-left: 64px !important; - } - - .m-lg-9 { - margin: 72px !important; - } - - .mt-lg-9, -.my-lg-9 { - margin-top: 72px !important; - } - - .mr-lg-9, -.mx-lg-9 { - margin-right: 72px !important; - } - - .mb-lg-9, -.my-lg-9 { - margin-bottom: 72px !important; - } - - .ml-lg-9, -.mx-lg-9 { - margin-left: 72px !important; - } - - .m-lg-10 { - margin: 80px !important; - } - - .mt-lg-10, -.my-lg-10 { - margin-top: 80px !important; - } - - .mr-lg-10, -.mx-lg-10 { - margin-right: 80px !important; - } - - .mb-lg-10, -.my-lg-10 { - margin-bottom: 80px !important; - } - - .ml-lg-10, -.mx-lg-10 { - margin-left: 80px !important; - } - - .m-lg-12 { - margin: 96px !important; - } - - .mt-lg-12, -.my-lg-12 { - margin-top: 96px !important; - } - - .mr-lg-12, -.mx-lg-12 { - margin-right: 96px !important; - } - - .mb-lg-12, -.my-lg-12 { - margin-bottom: 96px !important; - } - - .ml-lg-12, -.mx-lg-12 { - margin-left: 96px !important; - } - - .m-lg-15 { - margin: 120px !important; - } - - .mt-lg-15, -.my-lg-15 { - margin-top: 120px !important; - } - - .mr-lg-15, -.mx-lg-15 { - margin-right: 120px !important; - } - - .mb-lg-15, -.my-lg-15 { - margin-bottom: 120px !important; - } - - .ml-lg-15, -.mx-lg-15 { - margin-left: 120px !important; - } - - .p-lg-0 { - padding: 0 !important; - } - - .pt-lg-0, -.py-lg-0 { - padding-top: 0 !important; - } - - .pr-lg-0, -.px-lg-0 { - padding-right: 0 !important; - } - - .pb-lg-0, -.py-lg-0 { - padding-bottom: 0 !important; - } - - .pl-lg-0, -.px-lg-0 { - padding-left: 0 !important; - } - - .p-lg-1 { - padding: 8px !important; - } - - .pt-lg-1, -.py-lg-1 { - padding-top: 8px !important; - } - - .pr-lg-1, -.px-lg-1 { - padding-right: 8px !important; - } - - .pb-lg-1, -.py-lg-1 { - padding-bottom: 8px !important; - } - - .pl-lg-1, -.px-lg-1 { - padding-left: 8px !important; - } - - .p-lg-2 { - padding: 16px !important; - } - - .pt-lg-2, -.py-lg-2 { - padding-top: 16px !important; - } - - .pr-lg-2, -.px-lg-2 { - padding-right: 16px !important; - } - - .pb-lg-2, -.py-lg-2 { - padding-bottom: 16px !important; - } - - .pl-lg-2, -.px-lg-2 { - padding-left: 16px !important; - } - - .p-lg-3 { - padding: 24px !important; - } - - .pt-lg-3, -.py-lg-3 { - padding-top: 24px !important; - } - - .pr-lg-3, -.px-lg-3 { - padding-right: 24px !important; - } - - .pb-lg-3, -.py-lg-3 { - padding-bottom: 24px !important; - } - - .pl-lg-3, -.px-lg-3 { - padding-left: 24px !important; - } - - .p-lg-4 { - padding: 32px !important; - } - - .pt-lg-4, -.py-lg-4 { - padding-top: 32px !important; - } - - .pr-lg-4, -.px-lg-4 { - padding-right: 32px !important; - } - - .pb-lg-4, -.py-lg-4 { - padding-bottom: 32px !important; - } - - .pl-lg-4, -.px-lg-4 { - padding-left: 32px !important; - } - - .p-lg-5 { - padding: 40px !important; - } - - .pt-lg-5, -.py-lg-5 { - padding-top: 40px !important; - } - - .pr-lg-5, -.px-lg-5 { - padding-right: 40px !important; - } - - .pb-lg-5, -.py-lg-5 { - padding-bottom: 40px !important; - } - - .pl-lg-5, -.px-lg-5 { - padding-left: 40px !important; - } - - .p-lg-6 { - padding: 48px !important; - } - - .pt-lg-6, -.py-lg-6 { - padding-top: 48px !important; - } - - .pr-lg-6, -.px-lg-6 { - padding-right: 48px !important; - } - - .pb-lg-6, -.py-lg-6 { - padding-bottom: 48px !important; - } - - .pl-lg-6, -.px-lg-6 { - padding-left: 48px !important; - } - - .p-lg-7 { - padding: 56px !important; - } - - .pt-lg-7, -.py-lg-7 { - padding-top: 56px !important; - } - - .pr-lg-7, -.px-lg-7 { - padding-right: 56px !important; - } - - .pb-lg-7, -.py-lg-7 { - padding-bottom: 56px !important; - } - - .pl-lg-7, -.px-lg-7 { - padding-left: 56px !important; - } - - .p-lg-8 { - padding: 64px !important; - } - - .pt-lg-8, -.py-lg-8 { - padding-top: 64px !important; - } - - .pr-lg-8, -.px-lg-8 { - padding-right: 64px !important; - } - - .pb-lg-8, -.py-lg-8 { - padding-bottom: 64px !important; - } - - .pl-lg-8, -.px-lg-8 { - padding-left: 64px !important; - } - - .p-lg-9 { - padding: 72px !important; - } - - .pt-lg-9, -.py-lg-9 { - padding-top: 72px !important; - } - - .pr-lg-9, -.px-lg-9 { - padding-right: 72px !important; - } - - .pb-lg-9, -.py-lg-9 { - padding-bottom: 72px !important; - } - - .pl-lg-9, -.px-lg-9 { - padding-left: 72px !important; - } - - .p-lg-10 { - padding: 80px !important; - } - - .pt-lg-10, -.py-lg-10 { - padding-top: 80px !important; - } - - .pr-lg-10, -.px-lg-10 { - padding-right: 80px !important; - } - - .pb-lg-10, -.py-lg-10 { - padding-bottom: 80px !important; - } - - .pl-lg-10, -.px-lg-10 { - padding-left: 80px !important; - } - - .p-lg-12 { - padding: 96px !important; - } - - .pt-lg-12, -.py-lg-12 { - padding-top: 96px !important; - } - - .pr-lg-12, -.px-lg-12 { - padding-right: 96px !important; - } - - .pb-lg-12, -.py-lg-12 { - padding-bottom: 96px !important; - } - - .pl-lg-12, -.px-lg-12 { - padding-left: 96px !important; - } - - .p-lg-15 { - padding: 120px !important; - } - - .pt-lg-15, -.py-lg-15 { - padding-top: 120px !important; - } - - .pr-lg-15, -.px-lg-15 { - padding-right: 120px !important; - } - - .pb-lg-15, -.py-lg-15 { - padding-bottom: 120px !important; - } - - .pl-lg-15, -.px-lg-15 { - padding-left: 120px !important; - } - - .m-lg-n1 { - margin: -8px !important; - } - - .mt-lg-n1, -.my-lg-n1 { - margin-top: -8px !important; - } - - .mr-lg-n1, -.mx-lg-n1 { - margin-right: -8px !important; - } - - .mb-lg-n1, -.my-lg-n1 { - margin-bottom: -8px !important; - } - - .ml-lg-n1, -.mx-lg-n1 { - margin-left: -8px !important; - } - - .m-lg-n2 { - margin: -16px !important; - } - - .mt-lg-n2, -.my-lg-n2 { - margin-top: -16px !important; - } - - .mr-lg-n2, -.mx-lg-n2 { - margin-right: -16px !important; - } - - .mb-lg-n2, -.my-lg-n2 { - margin-bottom: -16px !important; - } - - .ml-lg-n2, -.mx-lg-n2 { - margin-left: -16px !important; - } - - .m-lg-n3 { - margin: -24px !important; - } - - .mt-lg-n3, -.my-lg-n3 { - margin-top: -24px !important; - } - - .mr-lg-n3, -.mx-lg-n3 { - margin-right: -24px !important; - } - - .mb-lg-n3, -.my-lg-n3 { - margin-bottom: -24px !important; - } - - .ml-lg-n3, -.mx-lg-n3 { - margin-left: -24px !important; - } - - .m-lg-n4 { - margin: -32px !important; - } - - .mt-lg-n4, -.my-lg-n4 { - margin-top: -32px !important; - } - - .mr-lg-n4, -.mx-lg-n4 { - margin-right: -32px !important; - } - - .mb-lg-n4, -.my-lg-n4 { - margin-bottom: -32px !important; - } - - .ml-lg-n4, -.mx-lg-n4 { - margin-left: -32px !important; - } - - .m-lg-n5 { - margin: -40px !important; - } - - .mt-lg-n5, -.my-lg-n5 { - margin-top: -40px !important; - } - - .mr-lg-n5, -.mx-lg-n5 { - margin-right: -40px !important; - } - - .mb-lg-n5, -.my-lg-n5 { - margin-bottom: -40px !important; - } - - .ml-lg-n5, -.mx-lg-n5 { - margin-left: -40px !important; - } - - .m-lg-n6 { - margin: -48px !important; - } - - .mt-lg-n6, -.my-lg-n6 { - margin-top: -48px !important; - } - - .mr-lg-n6, -.mx-lg-n6 { - margin-right: -48px !important; - } - - .mb-lg-n6, -.my-lg-n6 { - margin-bottom: -48px !important; - } - - .ml-lg-n6, -.mx-lg-n6 { - margin-left: -48px !important; - } - - .m-lg-n7 { - margin: -56px !important; - } - - .mt-lg-n7, -.my-lg-n7 { - margin-top: -56px !important; - } - - .mr-lg-n7, -.mx-lg-n7 { - margin-right: -56px !important; - } - - .mb-lg-n7, -.my-lg-n7 { - margin-bottom: -56px !important; - } - - .ml-lg-n7, -.mx-lg-n7 { - margin-left: -56px !important; - } - - .m-lg-n8 { - margin: -64px !important; - } - - .mt-lg-n8, -.my-lg-n8 { - margin-top: -64px !important; - } - - .mr-lg-n8, -.mx-lg-n8 { - margin-right: -64px !important; - } - - .mb-lg-n8, -.my-lg-n8 { - margin-bottom: -64px !important; - } - - .ml-lg-n8, -.mx-lg-n8 { - margin-left: -64px !important; - } - - .m-lg-n9 { - margin: -72px !important; - } - - .mt-lg-n9, -.my-lg-n9 { - margin-top: -72px !important; - } - - .mr-lg-n9, -.mx-lg-n9 { - margin-right: -72px !important; - } - - .mb-lg-n9, -.my-lg-n9 { - margin-bottom: -72px !important; - } - - .ml-lg-n9, -.mx-lg-n9 { - margin-left: -72px !important; - } - - .m-lg-n10 { - margin: -80px !important; - } - - .mt-lg-n10, -.my-lg-n10 { - margin-top: -80px !important; - } - - .mr-lg-n10, -.mx-lg-n10 { - margin-right: -80px !important; - } - - .mb-lg-n10, -.my-lg-n10 { - margin-bottom: -80px !important; - } - - .ml-lg-n10, -.mx-lg-n10 { - margin-left: -80px !important; - } - - .m-lg-n12 { - margin: -96px !important; - } - - .mt-lg-n12, -.my-lg-n12 { - margin-top: -96px !important; - } - - .mr-lg-n12, -.mx-lg-n12 { - margin-right: -96px !important; - } - - .mb-lg-n12, -.my-lg-n12 { - margin-bottom: -96px !important; - } - - .ml-lg-n12, -.mx-lg-n12 { - margin-left: -96px !important; - } - - .m-lg-n15 { - margin: -120px !important; - } - - .mt-lg-n15, -.my-lg-n15 { - margin-top: -120px !important; - } - - .mr-lg-n15, -.mx-lg-n15 { - margin-right: -120px !important; - } - - .mb-lg-n15, -.my-lg-n15 { - margin-bottom: -120px !important; - } - - .ml-lg-n15, -.mx-lg-n15 { - margin-left: -120px !important; - } - - .m-lg-auto { - margin: auto !important; - } - - .mt-lg-auto, -.my-lg-auto { - margin-top: auto !important; - } - - .mr-lg-auto, -.mx-lg-auto { - margin-right: auto !important; - } - - .mb-lg-auto, -.my-lg-auto { - margin-bottom: auto !important; - } - - .ml-lg-auto, -.mx-lg-auto { - margin-left: auto !important; - } -} -@media (min-width: 1240px) { - .m-xl-0 { - margin: 0 !important; - } - - .mt-xl-0, -.my-xl-0 { - margin-top: 0 !important; - } - - .mr-xl-0, -.mx-xl-0 { - margin-right: 0 !important; - } - - .mb-xl-0, -.my-xl-0 { - margin-bottom: 0 !important; - } - - .ml-xl-0, -.mx-xl-0 { - margin-left: 0 !important; - } - - .m-xl-1 { - margin: 8px !important; - } - - .mt-xl-1, -.my-xl-1 { - margin-top: 8px !important; - } - - .mr-xl-1, -.mx-xl-1 { - margin-right: 8px !important; - } - - .mb-xl-1, -.my-xl-1 { - margin-bottom: 8px !important; - } - - .ml-xl-1, -.mx-xl-1 { - margin-left: 8px !important; - } - - .m-xl-2 { - margin: 16px !important; - } - - .mt-xl-2, -.my-xl-2 { - margin-top: 16px !important; - } - - .mr-xl-2, -.mx-xl-2 { - margin-right: 16px !important; - } - - .mb-xl-2, -.my-xl-2 { - margin-bottom: 16px !important; - } - - .ml-xl-2, -.mx-xl-2 { - margin-left: 16px !important; - } - - .m-xl-3 { - margin: 24px !important; - } - - .mt-xl-3, -.my-xl-3 { - margin-top: 24px !important; - } - - .mr-xl-3, -.mx-xl-3 { - margin-right: 24px !important; - } - - .mb-xl-3, -.my-xl-3 { - margin-bottom: 24px !important; - } - - .ml-xl-3, -.mx-xl-3 { - margin-left: 24px !important; - } - - .m-xl-4 { - margin: 32px !important; - } - - .mt-xl-4, -.my-xl-4 { - margin-top: 32px !important; - } - - .mr-xl-4, -.mx-xl-4 { - margin-right: 32px !important; - } - - .mb-xl-4, -.my-xl-4 { - margin-bottom: 32px !important; - } - - .ml-xl-4, -.mx-xl-4 { - margin-left: 32px !important; - } - - .m-xl-5 { - margin: 40px !important; - } - - .mt-xl-5, -.my-xl-5 { - margin-top: 40px !important; - } - - .mr-xl-5, -.mx-xl-5 { - margin-right: 40px !important; - } - - .mb-xl-5, -.my-xl-5 { - margin-bottom: 40px !important; - } - - .ml-xl-5, -.mx-xl-5 { - margin-left: 40px !important; - } - - .m-xl-6 { - margin: 48px !important; - } - - .mt-xl-6, -.my-xl-6 { - margin-top: 48px !important; - } - - .mr-xl-6, -.mx-xl-6 { - margin-right: 48px !important; - } - - .mb-xl-6, -.my-xl-6 { - margin-bottom: 48px !important; - } - - .ml-xl-6, -.mx-xl-6 { - margin-left: 48px !important; - } - - .m-xl-7 { - margin: 56px !important; - } - - .mt-xl-7, -.my-xl-7 { - margin-top: 56px !important; - } - - .mr-xl-7, -.mx-xl-7 { - margin-right: 56px !important; - } - - .mb-xl-7, -.my-xl-7 { - margin-bottom: 56px !important; - } - - .ml-xl-7, -.mx-xl-7 { - margin-left: 56px !important; - } - - .m-xl-8 { - margin: 64px !important; - } - - .mt-xl-8, -.my-xl-8 { - margin-top: 64px !important; - } - - .mr-xl-8, -.mx-xl-8 { - margin-right: 64px !important; - } - - .mb-xl-8, -.my-xl-8 { - margin-bottom: 64px !important; - } - - .ml-xl-8, -.mx-xl-8 { - margin-left: 64px !important; - } - - .m-xl-9 { - margin: 72px !important; - } - - .mt-xl-9, -.my-xl-9 { - margin-top: 72px !important; - } - - .mr-xl-9, -.mx-xl-9 { - margin-right: 72px !important; - } - - .mb-xl-9, -.my-xl-9 { - margin-bottom: 72px !important; - } - - .ml-xl-9, -.mx-xl-9 { - margin-left: 72px !important; - } - - .m-xl-10 { - margin: 80px !important; - } - - .mt-xl-10, -.my-xl-10 { - margin-top: 80px !important; - } - - .mr-xl-10, -.mx-xl-10 { - margin-right: 80px !important; - } - - .mb-xl-10, -.my-xl-10 { - margin-bottom: 80px !important; - } - - .ml-xl-10, -.mx-xl-10 { - margin-left: 80px !important; - } - - .m-xl-12 { - margin: 96px !important; - } - - .mt-xl-12, -.my-xl-12 { - margin-top: 96px !important; - } - - .mr-xl-12, -.mx-xl-12 { - margin-right: 96px !important; - } - - .mb-xl-12, -.my-xl-12 { - margin-bottom: 96px !important; - } - - .ml-xl-12, -.mx-xl-12 { - margin-left: 96px !important; - } - - .m-xl-15 { - margin: 120px !important; - } - - .mt-xl-15, -.my-xl-15 { - margin-top: 120px !important; - } - - .mr-xl-15, -.mx-xl-15 { - margin-right: 120px !important; - } - - .mb-xl-15, -.my-xl-15 { - margin-bottom: 120px !important; - } - - .ml-xl-15, -.mx-xl-15 { - margin-left: 120px !important; - } - - .p-xl-0 { - padding: 0 !important; - } - - .pt-xl-0, -.py-xl-0 { - padding-top: 0 !important; - } - - .pr-xl-0, -.px-xl-0 { - padding-right: 0 !important; - } - - .pb-xl-0, -.py-xl-0 { - padding-bottom: 0 !important; - } - - .pl-xl-0, -.px-xl-0 { - padding-left: 0 !important; - } - - .p-xl-1 { - padding: 8px !important; - } - - .pt-xl-1, -.py-xl-1 { - padding-top: 8px !important; - } - - .pr-xl-1, -.px-xl-1 { - padding-right: 8px !important; - } - - .pb-xl-1, -.py-xl-1 { - padding-bottom: 8px !important; - } - - .pl-xl-1, -.px-xl-1 { - padding-left: 8px !important; - } - - .p-xl-2 { - padding: 16px !important; - } - - .pt-xl-2, -.py-xl-2 { - padding-top: 16px !important; - } - - .pr-xl-2, -.px-xl-2 { - padding-right: 16px !important; - } - - .pb-xl-2, -.py-xl-2 { - padding-bottom: 16px !important; - } - - .pl-xl-2, -.px-xl-2 { - padding-left: 16px !important; - } - - .p-xl-3 { - padding: 24px !important; - } - - .pt-xl-3, -.py-xl-3 { - padding-top: 24px !important; - } - - .pr-xl-3, -.px-xl-3 { - padding-right: 24px !important; - } - - .pb-xl-3, -.py-xl-3 { - padding-bottom: 24px !important; - } - - .pl-xl-3, -.px-xl-3 { - padding-left: 24px !important; - } - - .p-xl-4 { - padding: 32px !important; - } - - .pt-xl-4, -.py-xl-4 { - padding-top: 32px !important; - } - - .pr-xl-4, -.px-xl-4 { - padding-right: 32px !important; - } - - .pb-xl-4, -.py-xl-4 { - padding-bottom: 32px !important; - } - - .pl-xl-4, -.px-xl-4 { - padding-left: 32px !important; - } - - .p-xl-5 { - padding: 40px !important; - } - - .pt-xl-5, -.py-xl-5 { - padding-top: 40px !important; - } - - .pr-xl-5, -.px-xl-5 { - padding-right: 40px !important; - } - - .pb-xl-5, -.py-xl-5 { - padding-bottom: 40px !important; - } - - .pl-xl-5, -.px-xl-5 { - padding-left: 40px !important; - } - - .p-xl-6 { - padding: 48px !important; - } - - .pt-xl-6, -.py-xl-6 { - padding-top: 48px !important; - } - - .pr-xl-6, -.px-xl-6 { - padding-right: 48px !important; - } - - .pb-xl-6, -.py-xl-6 { - padding-bottom: 48px !important; - } - - .pl-xl-6, -.px-xl-6 { - padding-left: 48px !important; - } - - .p-xl-7 { - padding: 56px !important; - } - - .pt-xl-7, -.py-xl-7 { - padding-top: 56px !important; - } - - .pr-xl-7, -.px-xl-7 { - padding-right: 56px !important; - } - - .pb-xl-7, -.py-xl-7 { - padding-bottom: 56px !important; - } - - .pl-xl-7, -.px-xl-7 { - padding-left: 56px !important; - } - - .p-xl-8 { - padding: 64px !important; - } - - .pt-xl-8, -.py-xl-8 { - padding-top: 64px !important; - } - - .pr-xl-8, -.px-xl-8 { - padding-right: 64px !important; - } - - .pb-xl-8, -.py-xl-8 { - padding-bottom: 64px !important; - } - - .pl-xl-8, -.px-xl-8 { - padding-left: 64px !important; - } - - .p-xl-9 { - padding: 72px !important; - } - - .pt-xl-9, -.py-xl-9 { - padding-top: 72px !important; - } - - .pr-xl-9, -.px-xl-9 { - padding-right: 72px !important; - } - - .pb-xl-9, -.py-xl-9 { - padding-bottom: 72px !important; - } - - .pl-xl-9, -.px-xl-9 { - padding-left: 72px !important; - } - - .p-xl-10 { - padding: 80px !important; - } - - .pt-xl-10, -.py-xl-10 { - padding-top: 80px !important; - } - - .pr-xl-10, -.px-xl-10 { - padding-right: 80px !important; - } - - .pb-xl-10, -.py-xl-10 { - padding-bottom: 80px !important; - } - - .pl-xl-10, -.px-xl-10 { - padding-left: 80px !important; - } - - .p-xl-12 { - padding: 96px !important; - } - - .pt-xl-12, -.py-xl-12 { - padding-top: 96px !important; - } - - .pr-xl-12, -.px-xl-12 { - padding-right: 96px !important; - } - - .pb-xl-12, -.py-xl-12 { - padding-bottom: 96px !important; - } - - .pl-xl-12, -.px-xl-12 { - padding-left: 96px !important; - } - - .p-xl-15 { - padding: 120px !important; - } - - .pt-xl-15, -.py-xl-15 { - padding-top: 120px !important; - } - - .pr-xl-15, -.px-xl-15 { - padding-right: 120px !important; - } - - .pb-xl-15, -.py-xl-15 { - padding-bottom: 120px !important; - } - - .pl-xl-15, -.px-xl-15 { - padding-left: 120px !important; - } - - .m-xl-n1 { - margin: -8px !important; - } - - .mt-xl-n1, -.my-xl-n1 { - margin-top: -8px !important; - } - - .mr-xl-n1, -.mx-xl-n1 { - margin-right: -8px !important; - } - - .mb-xl-n1, -.my-xl-n1 { - margin-bottom: -8px !important; - } - - .ml-xl-n1, -.mx-xl-n1 { - margin-left: -8px !important; - } - - .m-xl-n2 { - margin: -16px !important; - } - - .mt-xl-n2, -.my-xl-n2 { - margin-top: -16px !important; - } - - .mr-xl-n2, -.mx-xl-n2 { - margin-right: -16px !important; - } - - .mb-xl-n2, -.my-xl-n2 { - margin-bottom: -16px !important; - } - - .ml-xl-n2, -.mx-xl-n2 { - margin-left: -16px !important; - } - - .m-xl-n3 { - margin: -24px !important; - } - - .mt-xl-n3, -.my-xl-n3 { - margin-top: -24px !important; - } - - .mr-xl-n3, -.mx-xl-n3 { - margin-right: -24px !important; - } - - .mb-xl-n3, -.my-xl-n3 { - margin-bottom: -24px !important; - } - - .ml-xl-n3, -.mx-xl-n3 { - margin-left: -24px !important; - } - - .m-xl-n4 { - margin: -32px !important; - } - - .mt-xl-n4, -.my-xl-n4 { - margin-top: -32px !important; - } - - .mr-xl-n4, -.mx-xl-n4 { - margin-right: -32px !important; - } - - .mb-xl-n4, -.my-xl-n4 { - margin-bottom: -32px !important; - } - - .ml-xl-n4, -.mx-xl-n4 { - margin-left: -32px !important; - } - - .m-xl-n5 { - margin: -40px !important; - } - - .mt-xl-n5, -.my-xl-n5 { - margin-top: -40px !important; - } - - .mr-xl-n5, -.mx-xl-n5 { - margin-right: -40px !important; - } - - .mb-xl-n5, -.my-xl-n5 { - margin-bottom: -40px !important; - } - - .ml-xl-n5, -.mx-xl-n5 { - margin-left: -40px !important; - } - - .m-xl-n6 { - margin: -48px !important; - } - - .mt-xl-n6, -.my-xl-n6 { - margin-top: -48px !important; - } - - .mr-xl-n6, -.mx-xl-n6 { - margin-right: -48px !important; - } - - .mb-xl-n6, -.my-xl-n6 { - margin-bottom: -48px !important; - } - - .ml-xl-n6, -.mx-xl-n6 { - margin-left: -48px !important; - } - - .m-xl-n7 { - margin: -56px !important; - } - - .mt-xl-n7, -.my-xl-n7 { - margin-top: -56px !important; - } - - .mr-xl-n7, -.mx-xl-n7 { - margin-right: -56px !important; - } - - .mb-xl-n7, -.my-xl-n7 { - margin-bottom: -56px !important; - } - - .ml-xl-n7, -.mx-xl-n7 { - margin-left: -56px !important; - } - - .m-xl-n8 { - margin: -64px !important; - } - - .mt-xl-n8, -.my-xl-n8 { - margin-top: -64px !important; - } - - .mr-xl-n8, -.mx-xl-n8 { - margin-right: -64px !important; - } - - .mb-xl-n8, -.my-xl-n8 { - margin-bottom: -64px !important; - } - - .ml-xl-n8, -.mx-xl-n8 { - margin-left: -64px !important; - } - - .m-xl-n9 { - margin: -72px !important; - } - - .mt-xl-n9, -.my-xl-n9 { - margin-top: -72px !important; - } - - .mr-xl-n9, -.mx-xl-n9 { - margin-right: -72px !important; - } - - .mb-xl-n9, -.my-xl-n9 { - margin-bottom: -72px !important; - } - - .ml-xl-n9, -.mx-xl-n9 { - margin-left: -72px !important; - } - - .m-xl-n10 { - margin: -80px !important; - } - - .mt-xl-n10, -.my-xl-n10 { - margin-top: -80px !important; - } - - .mr-xl-n10, -.mx-xl-n10 { - margin-right: -80px !important; - } - - .mb-xl-n10, -.my-xl-n10 { - margin-bottom: -80px !important; - } - - .ml-xl-n10, -.mx-xl-n10 { - margin-left: -80px !important; - } - - .m-xl-n12 { - margin: -96px !important; - } - - .mt-xl-n12, -.my-xl-n12 { - margin-top: -96px !important; - } - - .mr-xl-n12, -.mx-xl-n12 { - margin-right: -96px !important; - } - - .mb-xl-n12, -.my-xl-n12 { - margin-bottom: -96px !important; - } - - .ml-xl-n12, -.mx-xl-n12 { - margin-left: -96px !important; - } - - .m-xl-n15 { - margin: -120px !important; - } - - .mt-xl-n15, -.my-xl-n15 { - margin-top: -120px !important; - } - - .mr-xl-n15, -.mx-xl-n15 { - margin-right: -120px !important; - } - - .mb-xl-n15, -.my-xl-n15 { - margin-bottom: -120px !important; - } - - .ml-xl-n15, -.mx-xl-n15 { - margin-left: -120px !important; - } - - .m-xl-auto { - margin: auto !important; - } - - .mt-xl-auto, -.my-xl-auto { - margin-top: auto !important; - } - - .mr-xl-auto, -.mx-xl-auto { - margin-right: auto !important; - } - - .mb-xl-auto, -.my-xl-auto { - margin-bottom: auto !important; - } - - .ml-xl-auto, -.mx-xl-auto { - margin-left: auto !important; - } -} -.text-monospace { - font-family: SFMono-Regular, Menlo, Monaco, Consolas, "Liberation Mono", "Courier New", monospace !important; -} - -.text-justify { - text-align: justify !important; -} - -.text-wrap { - white-space: normal !important; -} - -.text-nowrap { - white-space: nowrap !important; -} - -.text-truncate { - overflow: hidden; - text-overflow: ellipsis; - white-space: nowrap; -} - -.text-left { - text-align: left !important; -} - -.text-right { - text-align: right !important; -} - -.text-center { - text-align: center !important; -} - -@media (min-width: 400px) { - .text-xs-left { - text-align: left !important; - } - - .text-xs-right { - text-align: right !important; - } - - .text-xs-center { - text-align: center !important; - } -} -@media (min-width: 616px) { - .text-sm-left { - text-align: left !important; - } - - .text-sm-right { - text-align: right !important; - } - - .text-sm-center { - text-align: center !important; - } -} -@media (min-width: 768px) { - .text-md-left { - text-align: left !important; - } - - .text-md-right { - text-align: right !important; - } - - .text-md-center { - text-align: center !important; - } -} -@media (min-width: 980px) { - .text-lg-left { - text-align: left !important; - } - - .text-lg-right { - text-align: right !important; - } - - .text-lg-center { - text-align: center !important; - } -} -@media (min-width: 1240px) { - .text-xl-left { - text-align: left !important; - } - - .text-xl-right { - text-align: right !important; - } - - .text-xl-center { - text-align: center !important; - } -} -.text-lowercase { - text-transform: lowercase !important; -} - -.text-uppercase { - text-transform: uppercase !important; -} - -.text-capitalize { - text-transform: capitalize !important; -} - -.font-weight-light { - font-weight: 300 !important; -} - -.font-weight-lighter { - font-weight: lighter !important; -} - -.font-weight-normal { - font-weight: 400 !important; -} - -.font-weight-bold { - font-weight: 700 !important; -} - -.font-weight-bolder { - font-weight: bolder !important; -} - -.font-italic { - font-style: italic !important; -} - -.text-white { - color: #fff !important; -} - -.text-primary { - color: #ffcc00 !important; -} - -a.text-primary:hover, a.text-primary:focus { - color: #b38f00 !important; -} - -.text-secondary { - color: #212529 !important; -} - -a.text-secondary:hover, a.text-secondary:focus { - color: black !important; -} - -.text-success { - color: #28a745 !important; -} - -a.text-success:hover, a.text-success:focus { - color: #19692c !important; -} - -.text-info { - color: #17a2b8 !important; -} - -a.text-info:hover, a.text-info:focus { - color: #0f6674 !important; -} - -.text-warning { - color: #ffc107 !important; -} - -a.text-warning:hover, a.text-warning:focus { - color: #ba8b00 !important; -} - -.text-danger { - color: #dc3545 !important; -} - -a.text-danger:hover, a.text-danger:focus { - color: #a71d2a !important; -} - -.text-light { - color: #f1f6f9 !important; -} - -a.text-light:hover, a.text-light:focus { - color: #bbd4e2 !important; -} - -.text-dark { - color: #495057 !important; -} - -a.text-dark:hover, a.text-dark:focus { - color: #262a2d !important; -} - -.text-primary-light { - color: #fffaf0 !important; -} - -a.text-primary-light:hover, a.text-primary-light:focus { - color: #ffe1a4 !important; -} - -.text-secondary-light { - color: #fff !important; -} - -a.text-secondary-light:hover, a.text-secondary-light:focus { - color: #d9d9d9 !important; -} - -.text-tertiary { - color: #257af4 !important; -} - -a.text-tertiary:hover, a.text-tertiary:focus { - color: #0a56c3 !important; -} - -.text-tertiary-light { - color: #e3f1fe !important; -} - -a.text-tertiary-light:hover, a.text-tertiary-light:focus { - color: #99ccfb !important; -} - -.text-white { - color: #fff !important; -} - -a.text-white:hover, a.text-white:focus { - color: #d9d9d9 !important; -} - -.text-black { - color: #212529 !important; -} - -a.text-black:hover, a.text-black:focus { - color: black !important; -} - -.text-blue { - color: #257af4 !important; -} - -a.text-blue:hover, a.text-blue:focus { - color: #0a56c3 !important; -} - -.text-light-blue { - color: #e3f1fe !important; -} - -a.text-light-blue:hover, a.text-light-blue:focus { - color: #99ccfb !important; -} - -.text-yellow { - color: #ffcc00 !important; -} - -a.text-yellow:hover, a.text-yellow:focus { - color: #b38f00 !important; -} - -.text-light-yellow { - color: #fffaf0 !important; -} - -a.text-light-yellow:hover, a.text-light-yellow:focus { - color: #ffe1a4 !important; -} - -.text-orange { - color: #ff8c00 !important; -} - -a.text-orange:hover, a.text-orange:focus { - color: #b36200 !important; -} - -.text-light-orange { - color: #ffe4b5 !important; -} - -a.text-light-orange:hover, a.text-light-orange:focus { - color: #ffc869 !important; -} - -.text-red { - color: #ff3939 !important; -} - -a.text-red:hover, a.text-red:focus { - color: #ec0000 !important; -} - -.text-light-red { - color: #ffe4e1 !important; -} - -a.text-light-red:hover, a.text-light-red:focus { - color: #ff9f95 !important; -} - -.text-medium { - color: #d6dbdf !important; -} - -a.text-medium:hover, a.text-medium:focus { - color: #abb5bd !important; -} - -.text-body { - color: #212529 !important; -} - -.text-muted { - color: #6c757d !important; -} - -.text-black-50 { - color: rgba(33, 37, 41, 0.5) !important; -} - -.text-white-50 { - color: rgba(255, 255, 255, 0.5) !important; -} - -.text-hide { - font: 0/0 a; - color: transparent; - text-shadow: none; - background-color: transparent; - border: 0; -} - -.text-decoration-none { - text-decoration: none !important; -} - -.text-break { - word-break: break-word !important; - overflow-wrap: break-word !important; -} - -.text-reset { - color: inherit !important; -} - -.visible { - visibility: visible !important; -} - -.invisible { - visibility: hidden !important; -} - -@media print { - *, -*::before, -*::after { - text-shadow: none !important; - box-shadow: none !important; - } - - a:not(.btn) { - text-decoration: underline; - } - - abbr[title]::after { - content: " (" attr(title) ")"; - } - - pre { - white-space: pre-wrap !important; - } - - pre, -blockquote { - border: 1px solid #d6dbdf; - page-break-inside: avoid; - } - - thead { - display: table-header-group; - } - - tr, -img { - page-break-inside: avoid; - } - - p, -h2, -h3 { - orphans: 3; - widows: 3; - } - - h2, -h3 { - page-break-after: avoid; - } - - @page { - size: a3; - } - body { - min-width: 980px !important; - } - - .container { - min-width: 980px !important; - } - - .navbar { - display: none; - } - - .badge { - border: 1px solid #212529; - } - - .table { - border-collapse: collapse !important; - } - .table td, -.table th { - background-color: #fff !important; - } - - .table-bordered th, -.table-bordered td { - border: 1px solid #dee2e6 !important; - } - - .table-dark { - color: inherit; - } - .table-dark th, -.table-dark td, -.table-dark thead th, -.table-dark tbody + tbody { - border-color: #d6dbdf; - } - - .table .thead-dark th { - color: inherit; - border-color: #d6dbdf; - } -} \ No newline at end of file + */:root{--blue:#007bff;--indigo:#6610f2;--purple:#6f42c1;--pink:#e83e8c;--red:#dc3545;--orange:#fd7e14;--yellow:#ffc107;--green:#28a745;--teal:#20c997;--cyan:#17a2b8;--gray:#6c757d;--gray-dark:#343a40;--brand-primary:#fc0;--brand-secondary:#ff3939;--primary-accent-yellow:#fc0;--primary-accent-light-yellow:#fffaf0;--primary-accent-blue:#257af4;--primary-accent-light-blue:#e3f1fe;--secondary-accent-orange:#ff8c00;--secondary-accent-light-orange:#ffe4b5;--secondary-accent-red:#ff3939;--secondary-accent-light-red:#ffe4e1;--primary:#fc0;--secondary:#212529;--success:#28a745;--info:#17a2b8;--warning:#ffc107;--danger:#dc3545;--light:#f1f6f9;--dark:#495057;--primary-light:#fffaf0;--secondary-light:#fff;--tertiary:#257af4;--tertiary-light:#e3f1fe;--white:#fff;--black:#212529;--blue:#257af4;--light-blue:#e3f1fe;--yellow:#fc0;--light-yellow:#fffaf0;--orange:#ff8c00;--light-orange:#ffe4b5;--red:#ff3939;--light-red:#ffe4e1;--medium:#d6dbdf;--breakpoint-xxs:0;--breakpoint-xs:400px;--breakpoint-sm:616px;--breakpoint-md:768px;--breakpoint-lg:980px;--breakpoint-xl:1240px;--font-family-sans-serif:"Noto Sans",sans-serif;--font-family-monospace:SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace}*,:after,:before{box-sizing:border-box}html{font-family:sans-serif;line-height:1.15;-webkit-text-size-adjust:100%;-webkit-tap-highlight-color:rgba(33,37,41,0)}article,aside,figcaption,figure,footer,header,hgroup,main,nav,section{display:block}body{margin:0;font-family:Noto Sans,sans-serif;font-size:1rem;font-weight:400;line-height:1.5;color:#212529;text-align:left;background-color:#fff}[tabindex="-1"]:focus:not(:focus-visible){outline:0!important}hr{box-sizing:content-box;height:0;overflow:visible}h1,h2,h3,h4,h5,h6{margin-top:0;margin-bottom:16px}p{margin-top:0;margin-bottom:1rem}abbr[data-original-title],abbr[title]{text-decoration:underline;-webkit-text-decoration:underline dotted;text-decoration:underline dotted;cursor:help;border-bottom:0;-webkit-text-decoration-skip-ink:none;text-decoration-skip-ink:none}address{font-style:normal;line-height:inherit}address,dl,ol,ul{margin-bottom:1rem}dl,ol,ul{margin-top:0}ol ol,ol ul,ul ol,ul ul{margin-bottom:0}dt{font-weight:700}dd{margin-bottom:.5rem;margin-left:0}blockquote{margin:0 0 1rem}b,strong{font-weight:bolder}small{font-size:80%}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sub{bottom:-.25em}sup{top:-.5em}a{text-decoration:none;background-color:transparent}a,a:hover{color:#ff8c00}a:hover{text-decoration:underline}a:not([href]),a:not([href]):hover{color:inherit;text-decoration:none}code,kbd,pre,samp{font-family:SFMono-Regular,Menlo,Monaco,Consolas,Liberation Mono,Courier New,monospace;font-size:1em}pre{margin-top:0;margin-bottom:1rem;overflow:auto}figure{margin:0 0 1rem}img{border-style:none}img,svg{vertical-align:middle}svg{overflow:hidden}table{border-collapse:collapse}caption{padding-top:.75rem;padding-bottom:.75rem;color:#6c757d;text-align:left;caption-side:bottom}th{text-align:inherit}label{display:inline-block;margin-bottom:.5rem}button{border-radius:0}button:focus{outline:1px dotted;outline:5px auto -webkit-focus-ring-color}button,input,optgroup,select,textarea{margin:0;font-family:inherit;font-size:inherit;line-height:inherit}button,input{overflow:visible}button,select{text-transform:none}select{word-wrap:normal}[type=button],[type=reset],[type=submit],button{-webkit-appearance:button}[type=button]:not(:disabled),[type=reset]:not(:disabled),[type=submit]:not(:disabled),button:not(:disabled){cursor:pointer}[type=button]::-moz-focus-inner,[type=reset]::-moz-focus-inner,[type=submit]::-moz-focus-inner,button::-moz-focus-inner{padding:0;border-style:none}input[type=checkbox],input[type=radio]{box-sizing:border-box;padding:0}input[type=date],input[type=datetime-local],input[type=month],input[type=time]{-webkit-appearance:listbox}textarea{overflow:auto;resize:vertical}fieldset{min-width:0;padding:0;margin:0;border:0}legend{display:block;width:100%;max-width:100%;padding:0;margin-bottom:.5rem;font-size:1.5rem;line-height:inherit;color:inherit;white-space:normal}@media(max-width:1200px){legend{font-size:calc(1.275rem + .3vw)}}progress{vertical-align:baseline}[type=number]::-webkit-inner-spin-button,[type=number]::-webkit-outer-spin-button{height:auto}[type=search]{outline-offset:-2px;-webkit-appearance:none}[type=search]::-webkit-search-decoration{-webkit-appearance:none}::-webkit-file-upload-button{font:inherit;-webkit-appearance:button}output{display:inline-block}summary{display:list-item;cursor:pointer}template{display:none}[hidden]{display:none!important}.h1,.h2,.h3,.h4,.h5,.h6,h1,h2,h3,h4,h5,h6{margin-bottom:16px;font-family:Hind Siliguri,sans-serif;font-weight:500;line-height:1.125}.h1,h1{font-size:2.5rem}@media(max-width:1200px){.h1,h1{font-size:calc(1.375rem + 1.5vw)}}.h2,h2{font-size:2rem}@media(max-width:1200px){.h2,h2{font-size:calc(1.325rem + .9vw)}}.h3,h3{font-size:1.75rem}@media(max-width:1200px){.h3,h3{font-size:calc(1.3rem + .6vw)}}.h4,h4{font-size:1.5rem}@media(max-width:1200px){.h4,h4{font-size:calc(1.275rem + .3vw)}}.h5,h5{font-size:1.125rem}.h6,h6{font-size:.875rem}.lead{font-size:1.375rem;font-weight:400}@media(max-width:1200px){.lead{font-size:calc(1.2625rem + .15vw)}}.display-1{font-size:4rem;font-weight:600;line-height:1.125}@media(max-width:1200px){.display-1{font-size:calc(1.525rem + 3.3vw)}}.display-2{font-size:2.5rem;font-weight:600;line-height:1.125}@media(max-width:1200px){.display-2{font-size:calc(1.375rem + 1.5vw)}}.display-3{font-size:2rem;font-weight:500;line-height:1.125}@media(max-width:1200px){.display-3{font-size:calc(1.325rem + .9vw)}}.display-4{font-size:1.75rem;font-weight:500;line-height:1.125}@media(max-width:1200px){.display-4{font-size:calc(1.3rem + .6vw)}}hr{margin-top:8px;margin-bottom:8px;border:0;border-top:1px solid rgba(33,37,41,.1)}.small,small{font-size:80%;font-weight:400}.mark,mark{padding:.2em;background-color:#fcf8e3}.list-inline,.list-unstyled{padding-left:0;list-style:none}.list-inline-item{display:inline-block}.list-inline-item:not(:last-child){margin-right:.5rem}.initialism{font-size:90%;text-transform:uppercase}.blockquote{margin-bottom:8px;font-size:1.25rem}.blockquote-footer{display:block;font-size:80%;color:#6c757d}.blockquote-footer:before{content:"— "}.img-fluid,.img-thumbnail{max-width:100%;height:auto}.img-thumbnail{padding:.25rem;background-color:#fff;border:1px solid #dee2e6;border-radius:8px}.figure{display:inline-block}.figure-img{margin-bottom:4px;line-height:1}.figure-caption{font-size:90%;color:#6c757d}code{font-size:87.5%;color:#e83e8c;word-wrap:break-word}a>code{color:inherit}kbd{padding:.2rem .4rem;font-size:87.5%;color:#fff;background-color:#495057;border-radius:8px}kbd kbd{padding:0;font-size:100%;font-weight:700}pre{display:block;font-size:87.5%;color:#495057}pre code{font-size:inherit;color:inherit;word-break:normal}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{width:100%;padding-right:20px;padding-left:20px;margin-right:auto;margin-left:auto}@media(min-width:400px){.container{max-width:576px}}@media(min-width:616px){.container{max-width:576px}}@media(min-width:768px){.container{max-width:958px}}@media(min-width:980px){.container{max-width:1008px}}@media(min-width:1240px){.container{max-width:1118px}}.container-fluid,.container-lg,.container-md,.container-sm,.container-xl,.container-xs{width:100%;padding-right:20px;padding-left:20px;margin-right:auto;margin-left:auto}@media(min-width:400px){.container,.container-xs{max-width:576px}}@media(min-width:616px){.container,.container-sm,.container-xs{max-width:576px}}@media(min-width:768px){.container,.container-md,.container-sm,.container-xs{max-width:958px}}@media(min-width:980px){.container,.container-lg,.container-md,.container-sm,.container-xs{max-width:1008px}}@media(min-width:1240px){.container,.container-lg,.container-md,.container-sm,.container-xl,.container-xs{max-width:1118px}}.row{display:flex;flex-wrap:wrap;margin-right:-20px;margin-left:-20px}.no-gutters{margin-right:0;margin-left:0}.no-gutters>.col,.no-gutters>[class*=col-]{padding-right:0;padding-left:0}.col,.col-1,.col-2,.col-3,.col-4,.col-5,.col-6,.col-7,.col-8,.col-9,.col-10,.col-11,.col-12,.col-auto,.col-lg,.col-lg-1,.col-lg-2,.col-lg-3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11,.col-lg-12,.col-lg-auto,.col-md,.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11,.col-md-12,.col-md-auto,.col-sm,.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11,.col-sm-12,.col-sm-auto,.col-xl,.col-xl-1,.col-xl-2,.col-xl-3,.col-xl-4,.col-xl-5,.col-xl-6,.col-xl-7,.col-xl-8,.col-xl-9,.col-xl-10,.col-xl-11,.col-xl-12,.col-xl-auto,.col-xs,.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11,.col-xs-12,.col-xs-auto{position:relative;width:100%;padding-right:20px;padding-left:20px}.col{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-1>*{flex:0 0 100%;max-width:100%}.row-cols-2>*{flex:0 0 50%;max-width:50%}.row-cols-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-4>*{flex:0 0 25%;max-width:25%}.row-cols-5>*{flex:0 0 20%;max-width:20%}.row-cols-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-auto{flex:0 0 auto;width:auto;max-width:100%}.col-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-3{flex:0 0 25%;max-width:25%}.col-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-6{flex:0 0 50%;max-width:50%}.col-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-9{flex:0 0 75%;max-width:75%}.col-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-12{flex:0 0 100%;max-width:100%}.order-first{order:-1}.order-last{order:13}.order-0{order:0}.order-1{order:1}.order-2{order:2}.order-3{order:3}.order-4{order:4}.order-5{order:5}.order-6{order:6}.order-7{order:7}.order-8{order:8}.order-9{order:9}.order-10{order:10}.order-11{order:11}.order-12{order:12}.offset-1{margin-left:8.3333333333%}.offset-2{margin-left:16.6666666667%}.offset-3{margin-left:25%}.offset-4{margin-left:33.3333333333%}.offset-5{margin-left:41.6666666667%}.offset-6{margin-left:50%}.offset-7{margin-left:58.3333333333%}.offset-8{margin-left:66.6666666667%}.offset-9{margin-left:75%}.offset-10{margin-left:83.3333333333%}.offset-11{margin-left:91.6666666667%}@media(min-width:400px){.col-xs{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-xs-1>*{flex:0 0 100%;max-width:100%}.row-cols-xs-2>*{flex:0 0 50%;max-width:50%}.row-cols-xs-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-xs-4>*{flex:0 0 25%;max-width:25%}.row-cols-xs-5>*{flex:0 0 20%;max-width:20%}.row-cols-xs-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xs-auto{flex:0 0 auto;width:auto;max-width:100%}.col-xs-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-xs-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xs-3{flex:0 0 25%;max-width:25%}.col-xs-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-xs-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-xs-6{flex:0 0 50%;max-width:50%}.col-xs-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-xs-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-xs-9{flex:0 0 75%;max-width:75%}.col-xs-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-xs-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-xs-12{flex:0 0 100%;max-width:100%}.order-xs-first{order:-1}.order-xs-last{order:13}.order-xs-0{order:0}.order-xs-1{order:1}.order-xs-2{order:2}.order-xs-3{order:3}.order-xs-4{order:4}.order-xs-5{order:5}.order-xs-6{order:6}.order-xs-7{order:7}.order-xs-8{order:8}.order-xs-9{order:9}.order-xs-10{order:10}.order-xs-11{order:11}.order-xs-12{order:12}.offset-xs-0{margin-left:0}.offset-xs-1{margin-left:8.3333333333%}.offset-xs-2{margin-left:16.6666666667%}.offset-xs-3{margin-left:25%}.offset-xs-4{margin-left:33.3333333333%}.offset-xs-5{margin-left:41.6666666667%}.offset-xs-6{margin-left:50%}.offset-xs-7{margin-left:58.3333333333%}.offset-xs-8{margin-left:66.6666666667%}.offset-xs-9{margin-left:75%}.offset-xs-10{margin-left:83.3333333333%}.offset-xs-11{margin-left:91.6666666667%}}@media(min-width:616px){.col-sm{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-sm-1>*{flex:0 0 100%;max-width:100%}.row-cols-sm-2>*{flex:0 0 50%;max-width:50%}.row-cols-sm-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-sm-4>*{flex:0 0 25%;max-width:25%}.row-cols-sm-5>*{flex:0 0 20%;max-width:20%}.row-cols-sm-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-sm-auto{flex:0 0 auto;width:auto;max-width:100%}.col-sm-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-sm-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-sm-3{flex:0 0 25%;max-width:25%}.col-sm-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-sm-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-sm-6{flex:0 0 50%;max-width:50%}.col-sm-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-sm-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-sm-9{flex:0 0 75%;max-width:75%}.col-sm-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-sm-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-sm-12{flex:0 0 100%;max-width:100%}.order-sm-first{order:-1}.order-sm-last{order:13}.order-sm-0{order:0}.order-sm-1{order:1}.order-sm-2{order:2}.order-sm-3{order:3}.order-sm-4{order:4}.order-sm-5{order:5}.order-sm-6{order:6}.order-sm-7{order:7}.order-sm-8{order:8}.order-sm-9{order:9}.order-sm-10{order:10}.order-sm-11{order:11}.order-sm-12{order:12}.offset-sm-0{margin-left:0}.offset-sm-1{margin-left:8.3333333333%}.offset-sm-2{margin-left:16.6666666667%}.offset-sm-3{margin-left:25%}.offset-sm-4{margin-left:33.3333333333%}.offset-sm-5{margin-left:41.6666666667%}.offset-sm-6{margin-left:50%}.offset-sm-7{margin-left:58.3333333333%}.offset-sm-8{margin-left:66.6666666667%}.offset-sm-9{margin-left:75%}.offset-sm-10{margin-left:83.3333333333%}.offset-sm-11{margin-left:91.6666666667%}}@media(min-width:768px){.col-md{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-md-1>*{flex:0 0 100%;max-width:100%}.row-cols-md-2>*{flex:0 0 50%;max-width:50%}.row-cols-md-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-md-4>*{flex:0 0 25%;max-width:25%}.row-cols-md-5>*{flex:0 0 20%;max-width:20%}.row-cols-md-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-md-auto{flex:0 0 auto;width:auto;max-width:100%}.col-md-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-md-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-md-3{flex:0 0 25%;max-width:25%}.col-md-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-md-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-md-6{flex:0 0 50%;max-width:50%}.col-md-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-md-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-md-9{flex:0 0 75%;max-width:75%}.col-md-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-md-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-md-12{flex:0 0 100%;max-width:100%}.order-md-first{order:-1}.order-md-last{order:13}.order-md-0{order:0}.order-md-1{order:1}.order-md-2{order:2}.order-md-3{order:3}.order-md-4{order:4}.order-md-5{order:5}.order-md-6{order:6}.order-md-7{order:7}.order-md-8{order:8}.order-md-9{order:9}.order-md-10{order:10}.order-md-11{order:11}.order-md-12{order:12}.offset-md-0{margin-left:0}.offset-md-1{margin-left:8.3333333333%}.offset-md-2{margin-left:16.6666666667%}.offset-md-3{margin-left:25%}.offset-md-4{margin-left:33.3333333333%}.offset-md-5{margin-left:41.6666666667%}.offset-md-6{margin-left:50%}.offset-md-7{margin-left:58.3333333333%}.offset-md-8{margin-left:66.6666666667%}.offset-md-9{margin-left:75%}.offset-md-10{margin-left:83.3333333333%}.offset-md-11{margin-left:91.6666666667%}}@media(min-width:980px){.col-lg{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-lg-1>*{flex:0 0 100%;max-width:100%}.row-cols-lg-2>*{flex:0 0 50%;max-width:50%}.row-cols-lg-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-lg-4>*{flex:0 0 25%;max-width:25%}.row-cols-lg-5>*{flex:0 0 20%;max-width:20%}.row-cols-lg-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-lg-auto{flex:0 0 auto;width:auto;max-width:100%}.col-lg-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-lg-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-lg-3{flex:0 0 25%;max-width:25%}.col-lg-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-lg-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-lg-6{flex:0 0 50%;max-width:50%}.col-lg-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-lg-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-lg-9{flex:0 0 75%;max-width:75%}.col-lg-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-lg-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-lg-12{flex:0 0 100%;max-width:100%}.order-lg-first{order:-1}.order-lg-last{order:13}.order-lg-0{order:0}.order-lg-1{order:1}.order-lg-2{order:2}.order-lg-3{order:3}.order-lg-4{order:4}.order-lg-5{order:5}.order-lg-6{order:6}.order-lg-7{order:7}.order-lg-8{order:8}.order-lg-9{order:9}.order-lg-10{order:10}.order-lg-11{order:11}.order-lg-12{order:12}.offset-lg-0{margin-left:0}.offset-lg-1{margin-left:8.3333333333%}.offset-lg-2{margin-left:16.6666666667%}.offset-lg-3{margin-left:25%}.offset-lg-4{margin-left:33.3333333333%}.offset-lg-5{margin-left:41.6666666667%}.offset-lg-6{margin-left:50%}.offset-lg-7{margin-left:58.3333333333%}.offset-lg-8{margin-left:66.6666666667%}.offset-lg-9{margin-left:75%}.offset-lg-10{margin-left:83.3333333333%}.offset-lg-11{margin-left:91.6666666667%}}@media(min-width:1240px){.col-xl{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-xl-1>*{flex:0 0 100%;max-width:100%}.row-cols-xl-2>*{flex:0 0 50%;max-width:50%}.row-cols-xl-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-xl-4>*{flex:0 0 25%;max-width:25%}.row-cols-xl-5>*{flex:0 0 20%;max-width:20%}.row-cols-xl-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xl-auto{flex:0 0 auto;width:auto;max-width:100%}.col-xl-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-xl-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xl-3{flex:0 0 25%;max-width:25%}.col-xl-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-xl-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-xl-6{flex:0 0 50%;max-width:50%}.col-xl-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-xl-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-xl-9{flex:0 0 75%;max-width:75%}.col-xl-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-xl-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-xl-12{flex:0 0 100%;max-width:100%}.order-xl-first{order:-1}.order-xl-last{order:13}.order-xl-0{order:0}.order-xl-1{order:1}.order-xl-2{order:2}.order-xl-3{order:3}.order-xl-4{order:4}.order-xl-5{order:5}.order-xl-6{order:6}.order-xl-7{order:7}.order-xl-8{order:8}.order-xl-9{order:9}.order-xl-10{order:10}.order-xl-11{order:11}.order-xl-12{order:12}.offset-xl-0{margin-left:0}.offset-xl-1{margin-left:8.3333333333%}.offset-xl-2{margin-left:16.6666666667%}.offset-xl-3{margin-left:25%}.offset-xl-4{margin-left:33.3333333333%}.offset-xl-5{margin-left:41.6666666667%}.offset-xl-6{margin-left:50%}.offset-xl-7{margin-left:58.3333333333%}.offset-xl-8{margin-left:66.6666666667%}.offset-xl-9{margin-left:75%}.offset-xl-10{margin-left:83.3333333333%}.offset-xl-11{margin-left:91.6666666667%}}.table{width:100%;margin-bottom:8px;color:#212529}.table td,.table th{padding:.75rem;vertical-align:top;border-top:1px solid #d6dbdf}.table thead th{vertical-align:bottom;border-bottom:2px solid #d6dbdf}.table tbody+tbody{border-top:2px solid #d6dbdf}.table-sm td,.table-sm th{padding:.3rem}.table-bordered,.table-bordered td,.table-bordered th{border:1px solid #d6dbdf}.table-bordered thead td,.table-bordered thead th{border-bottom-width:2px}.table-borderless tbody+tbody,.table-borderless td,.table-borderless th,.table-borderless thead th{border:0}.table-striped tbody tr:nth-of-type(odd){background-color:rgba(33,37,41,.05)}.table-hover tbody tr:hover{color:#212529;background-color:rgba(33,37,41,.075)}.table-primary,.table-primary>td,.table-primary>th{background-color:#fff1b8}.table-primary tbody+tbody,.table-primary td,.table-primary th,.table-primary thead th{border-color:#ffe47a}.table-hover .table-primary:hover,.table-hover .table-primary:hover>td,.table-hover .table-primary:hover>th{background-color:#ffec9f}.table-secondary,.table-secondary>td,.table-secondary>th{background-color:#c1c2c3}.table-secondary tbody+tbody,.table-secondary td,.table-secondary th,.table-secondary thead th{border-color:#8c8e90}.table-hover .table-secondary:hover,.table-hover .table-secondary:hover>td,.table-hover .table-secondary:hover>th{background-color:#b4b5b6}.table-success,.table-success>td,.table-success>th{background-color:#c3e6cb}.table-success tbody+tbody,.table-success td,.table-success th,.table-success thead th{border-color:#8fd19e}.table-hover .table-success:hover,.table-hover .table-success:hover>td,.table-hover .table-success:hover>th{background-color:#b1dfbb}.table-info,.table-info>td,.table-info>th{background-color:#bee5eb}.table-info tbody+tbody,.table-info td,.table-info th,.table-info thead th{border-color:#86cfda}.table-hover .table-info:hover,.table-hover .table-info:hover>td,.table-hover .table-info:hover>th{background-color:#abdde5}.table-warning,.table-warning>td,.table-warning>th{background-color:#ffeeba}.table-warning tbody+tbody,.table-warning td,.table-warning th,.table-warning thead th{border-color:#ffdf7e}.table-hover .table-warning:hover,.table-hover .table-warning:hover>td,.table-hover .table-warning:hover>th{background-color:#ffe8a1}.table-danger,.table-danger>td,.table-danger>th{background-color:#f5c6cb}.table-danger tbody+tbody,.table-danger td,.table-danger th,.table-danger thead th{border-color:#ed969e}.table-hover .table-danger:hover,.table-hover .table-danger:hover>td,.table-hover .table-danger:hover>th{background-color:#f1b0b7}.table-light,.table-light>td,.table-light>th{background-color:#fbfcfd}.table-light tbody+tbody,.table-light td,.table-light th,.table-light thead th{border-color:#f8fafc}.table-hover .table-light:hover,.table-hover .table-light:hover>td,.table-hover .table-light:hover>th{background-color:#eaeff5}.table-dark,.table-dark>td,.table-dark>th{background-color:#ccced0}.table-dark tbody+tbody,.table-dark td,.table-dark th,.table-dark thead th{border-color:#a0a4a8}.table-hover .table-dark:hover,.table-hover .table-dark:hover>td,.table-hover .table-dark:hover>th{background-color:#bfc1c4}.table-primary-light,.table-primary-light>td,.table-primary-light>th{background-color:#fffefb}.table-primary-light tbody+tbody,.table-primary-light td,.table-primary-light th,.table-primary-light thead th{border-color:#fffcf7}.table-hover .table-primary-light:hover,.table-hover .table-primary-light:hover>td,.table-hover .table-primary-light:hover>th{background-color:#fff8e2}.table-secondary-light,.table-secondary-light>td,.table-secondary-light>th{background-color:#fff}.table-secondary-light tbody+tbody,.table-secondary-light td,.table-secondary-light th,.table-secondary-light thead th{border-color:#fff}.table-hover .table-secondary-light:hover,.table-hover .table-secondary-light:hover>td,.table-hover .table-secondary-light:hover>th{background-color:#f2f2f2}.table-tertiary,.table-tertiary>td,.table-tertiary>th{background-color:#c2dafc}.table-tertiary tbody+tbody,.table-tertiary td,.table-tertiary th,.table-tertiary thead th{border-color:#8ebaf9}.table-hover .table-tertiary:hover,.table-hover .table-tertiary:hover>td,.table-hover .table-tertiary:hover>th{background-color:#aacbfb}.table-tertiary-light,.table-tertiary-light>td,.table-tertiary-light>th{background-color:#f7fbff}.table-tertiary-light tbody+tbody,.table-tertiary-light td,.table-tertiary-light th,.table-tertiary-light thead th{border-color:#f0f8fe}.table-hover .table-tertiary-light:hover,.table-hover .table-tertiary-light:hover>td,.table-hover .table-tertiary-light:hover>th{background-color:#deeeff}.table-white,.table-white>td,.table-white>th{background-color:#fff}.table-white tbody+tbody,.table-white td,.table-white th,.table-white thead th{border-color:#fff}.table-hover .table-white:hover,.table-hover .table-white:hover>td,.table-hover .table-white:hover>th{background-color:#f2f2f2}.table-black,.table-black>td,.table-black>th{background-color:#c1c2c3}.table-black tbody+tbody,.table-black td,.table-black th,.table-black thead th{border-color:#8c8e90}.table-hover .table-black:hover,.table-hover .table-black:hover>td,.table-hover .table-black:hover>th{background-color:#b4b5b6}.table-blue,.table-blue>td,.table-blue>th{background-color:#c2dafc}.table-blue tbody+tbody,.table-blue td,.table-blue th,.table-blue thead th{border-color:#8ebaf9}.table-hover .table-blue:hover,.table-hover .table-blue:hover>td,.table-hover .table-blue:hover>th{background-color:#aacbfb}.table-light-blue,.table-light-blue>td,.table-light-blue>th{background-color:#f7fbff}.table-light-blue tbody+tbody,.table-light-blue td,.table-light-blue th,.table-light-blue thead th{border-color:#f0f8fe}.table-hover .table-light-blue:hover,.table-hover .table-light-blue:hover>td,.table-hover .table-light-blue:hover>th{background-color:#deeeff}.table-yellow,.table-yellow>td,.table-yellow>th{background-color:#fff1b8}.table-yellow tbody+tbody,.table-yellow td,.table-yellow th,.table-yellow thead th{border-color:#ffe47a}.table-hover .table-yellow:hover,.table-hover .table-yellow:hover>td,.table-hover .table-yellow:hover>th{background-color:#ffec9f}.table-light-yellow,.table-light-yellow>td,.table-light-yellow>th{background-color:#fffefb}.table-light-yellow tbody+tbody,.table-light-yellow td,.table-light-yellow th,.table-light-yellow thead th{border-color:#fffcf7}.table-hover .table-light-yellow:hover,.table-hover .table-light-yellow:hover>td,.table-hover .table-light-yellow:hover>th{background-color:#fff8e2}.table-orange,.table-orange>td,.table-orange>th{background-color:#ffdfb8}.table-orange tbody+tbody,.table-orange td,.table-orange th,.table-orange thead th{border-color:#ffc37a}.table-hover .table-orange:hover,.table-hover .table-orange:hover>td,.table-hover .table-orange:hover>th{background-color:#ffd49f}.table-light-orange,.table-light-orange>td,.table-light-orange>th{background-color:#fff7ea}.table-light-orange tbody+tbody,.table-light-orange td,.table-light-orange th,.table-light-orange thead th{border-color:#fff1d9}.table-hover .table-light-orange:hover,.table-hover .table-light-orange:hover>td,.table-hover .table-light-orange:hover>th{background-color:#ffedd1}.table-red,.table-red>td,.table-red>th{background-color:#ffc8c8}.table-red tbody+tbody,.table-red td,.table-red th,.table-red thead th{border-color:#ff9898}.table-hover .table-red:hover,.table-hover .table-red:hover>td,.table-hover .table-red:hover>th{background-color:#ffafaf}.table-light-red,.table-light-red>td,.table-light-red>th{background-color:#fff7f7}.table-light-red tbody+tbody,.table-light-red td,.table-light-red th,.table-light-red thead th{border-color:#fff1ef}.table-hover .table-light-red:hover,.table-hover .table-light-red:hover>td,.table-hover .table-light-red:hover>th{background-color:#ffdede}.table-medium,.table-medium>td,.table-medium>th{background-color:#f4f5f6}.table-medium tbody+tbody,.table-medium td,.table-medium th,.table-medium thead th{border-color:#eaecee}.table-hover .table-medium:hover,.table-hover .table-medium:hover>td,.table-hover .table-medium:hover>th{background-color:#e6e8eb}.table-active,.table-active>td,.table-active>th{background-color:rgba(33,37,41,.075)}.table-hover .table-active:hover,.table-hover .table-active:hover>td,.table-hover .table-active:hover>th{background-color:rgba(22,24,27,.075)}.table .thead-dark th{color:#fff;background-color:#343a40;border-color:#454d55}.table .thead-light th{color:#6c757d;background-color:#e9ecef;border-color:#d6dbdf}.table-dark{color:#fff;background-color:#343a40}.table-dark td,.table-dark th,.table-dark thead th{border-color:#454d55}.table-dark.table-bordered{border:0}.table-dark.table-striped tbody tr:nth-of-type(odd){background-color:hsla(0,0%,100%,.05)}.table-dark.table-hover tbody tr:hover{color:#fff;background-color:hsla(0,0%,100%,.075)}@media(max-width:399.98px){.table-responsive-xs{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-xs>.table-bordered{border:0}}@media(max-width:615.98px){.table-responsive-sm{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-sm>.table-bordered{border:0}}@media(max-width:767.98px){.table-responsive-md{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-md>.table-bordered{border:0}}@media(max-width:979.98px){.table-responsive-lg{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-lg>.table-bordered{border:0}}@media(max-width:1239.98px){.table-responsive-xl{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-xl>.table-bordered{border:0}}.table-responsive{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive>.table-bordered{border:0}.form-control{display:block;width:100%;height:calc(1.5em + .75rem + 2px);padding:.375rem .75rem;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;background-color:#fff;background-clip:padding-box;border:1px solid #ced4da;border-radius:8px;transition:border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.form-control{transition:none}}.form-control::-ms-expand{background-color:transparent;border:0}.form-control:-moz-focusring{color:transparent;text-shadow:0 0 0 #6c757d}.form-control:focus{color:#6c757d;background-color:#fff;border-color:#ffe680;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.form-control::-moz-placeholder{color:#6c757d;opacity:1}.form-control:-ms-input-placeholder{color:#6c757d;opacity:1}.form-control::placeholder{color:#6c757d;opacity:1}.form-control:disabled,.form-control[readonly]{background-color:#e9ecef;opacity:1}select.form-control:focus::-ms-value{color:#6c757d;background-color:#fff}.form-control-file,.form-control-range{display:block;width:100%}.col-form-label{padding-top:calc(.375rem + 1px);padding-bottom:calc(.375rem + 1px);margin-bottom:0;font-size:inherit;line-height:1.5}.col-form-label-lg{padding-top:calc(.5rem + 1px);padding-bottom:calc(.5rem + 1px);font-size:1.125rem;line-height:1.5}.col-form-label-sm{padding-top:calc(.25rem + 1px);padding-bottom:calc(.25rem + 1px);font-size:.875rem;line-height:1.5}.form-control-plaintext{display:block;width:100%;padding:.375rem 0;margin-bottom:0;font-size:1rem;line-height:1.5;color:#212529;background-color:transparent;border:solid transparent;border-width:1px 0}.form-control-plaintext.form-control-lg,.form-control-plaintext.form-control-sm{padding-right:0;padding-left:0}.form-control-sm{height:calc(1.5em + .5rem + 2px);padding:.25rem .5rem;font-size:.875rem;line-height:1.5;border-radius:8px}.form-control-lg{height:calc(1.5em + 1rem + 2px);padding:.5rem 1rem;font-size:1.125rem;line-height:1.5;border-radius:8px}select.form-control[multiple],select.form-control[size],textarea.form-control{height:auto}.form-group{margin-bottom:1rem}.form-text{display:block;margin-top:.25rem}.form-row{display:flex;flex-wrap:wrap;margin-right:-5px;margin-left:-5px}.form-row>.col,.form-row>[class*=col-]{padding-right:5px;padding-left:5px}.form-check{position:relative;display:block;padding-left:1.25rem}.form-check-input{position:absolute;margin-top:.3rem;margin-left:-1.25rem}.form-check-input:disabled~.form-check-label,.form-check-input[disabled]~.form-check-label{color:#6c757d}.form-check-label{margin-bottom:0}.form-check-inline{display:inline-flex;align-items:center;padding-left:0;margin-right:.75rem}.form-check-inline .form-check-input{position:static;margin-top:0;margin-right:.3125rem;margin-left:0}.valid-feedback{display:none;width:100%;margin-top:.25rem;font-size:80%;color:#28a745}.valid-tooltip{position:absolute;top:100%;z-index:5;display:none;max-width:100%;padding:.25rem .5rem;margin-top:.1rem;font-size:.875rem;line-height:1.5;color:#fff;background-color:rgba(40,167,69,.9);border-radius:8px}.is-valid~.valid-feedback,.is-valid~.valid-tooltip,.was-validated :valid~.valid-feedback,.was-validated :valid~.valid-tooltip{display:block}.form-control.is-valid,.was-validated .form-control:valid{border-color:#28a745;padding-right:calc(1.5em + .75rem);background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3E%3C/svg%3E");background-repeat:no-repeat;background-position:right calc(.375em + .1875rem) center;background-size:calc(.75em + .375rem) calc(.75em + .375rem)}.form-control.is-valid:focus,.was-validated .form-control:valid:focus{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.was-validated textarea.form-control:valid,textarea.form-control.is-valid{padding-right:calc(1.5em + .75rem);background-position:top calc(.375em + .1875rem) right calc(.375em + .1875rem)}.custom-select.is-valid,.was-validated .custom-select:valid{border-color:#28a745;padding-right:calc(.75em + 2.3125rem);background:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px,url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3E%3C/svg%3E") #fff no-repeat center right 1.75rem/calc(.75em + .375rem) calc(.75em + .375rem)}.custom-select.is-valid:focus,.was-validated .custom-select:valid:focus{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.form-check-input.is-valid~.form-check-label,.was-validated .form-check-input:valid~.form-check-label{color:#28a745}.form-check-input.is-valid~.valid-feedback,.form-check-input.is-valid~.valid-tooltip,.was-validated .form-check-input:valid~.valid-feedback,.was-validated .form-check-input:valid~.valid-tooltip{display:block}.custom-control-input.is-valid~.custom-control-label,.was-validated .custom-control-input:valid~.custom-control-label{color:#28a745}.custom-control-input.is-valid~.custom-control-label:before,.was-validated .custom-control-input:valid~.custom-control-label:before{border-color:#28a745}.custom-control-input.is-valid:checked~.custom-control-label:before,.was-validated .custom-control-input:valid:checked~.custom-control-label:before{border-color:#34ce57;background-color:#34ce57}.custom-control-input.is-valid:focus~.custom-control-label:before,.was-validated .custom-control-input:valid:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.custom-control-input.is-valid:focus:not(:checked)~.custom-control-label:before,.custom-file-input.is-valid~.custom-file-label,.was-validated .custom-control-input:valid:focus:not(:checked)~.custom-control-label:before,.was-validated .custom-file-input:valid~.custom-file-label{border-color:#28a745}.custom-file-input.is-valid:focus~.custom-file-label,.was-validated .custom-file-input:valid:focus~.custom-file-label{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.invalid-feedback{display:none;width:100%;margin-top:.25rem;font-size:80%;color:#dc3545}.invalid-tooltip{position:absolute;top:100%;z-index:5;display:none;max-width:100%;padding:.25rem .5rem;margin-top:.1rem;font-size:.875rem;line-height:1.5;color:#fff;background-color:rgba(220,53,69,.9);border-radius:8px}.is-invalid~.invalid-feedback,.is-invalid~.invalid-tooltip,.was-validated :invalid~.invalid-feedback,.was-validated :invalid~.invalid-tooltip{display:block}.form-control.is-invalid,.was-validated .form-control:invalid{border-color:#dc3545;padding-right:calc(1.5em + .75rem);background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545'%3E%3Ccircle cx='6' cy='6' r='4.5'/%3E%3Cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3E%3Ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3E%3C/svg%3E");background-repeat:no-repeat;background-position:right calc(.375em + .1875rem) center;background-size:calc(.75em + .375rem) calc(.75em + .375rem)}.form-control.is-invalid:focus,.was-validated .form-control:invalid:focus{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.was-validated textarea.form-control:invalid,textarea.form-control.is-invalid{padding-right:calc(1.5em + .75rem);background-position:top calc(.375em + .1875rem) right calc(.375em + .1875rem)}.custom-select.is-invalid,.was-validated .custom-select:invalid{border-color:#dc3545;padding-right:calc(.75em + 2.3125rem);background:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px,url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545'%3E%3Ccircle cx='6' cy='6' r='4.5'/%3E%3Cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3E%3Ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3E%3C/svg%3E") #fff no-repeat center right 1.75rem/calc(.75em + .375rem) calc(.75em + .375rem)}.custom-select.is-invalid:focus,.was-validated .custom-select:invalid:focus{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.form-check-input.is-invalid~.form-check-label,.was-validated .form-check-input:invalid~.form-check-label{color:#dc3545}.form-check-input.is-invalid~.invalid-feedback,.form-check-input.is-invalid~.invalid-tooltip,.was-validated .form-check-input:invalid~.invalid-feedback,.was-validated .form-check-input:invalid~.invalid-tooltip{display:block}.custom-control-input.is-invalid~.custom-control-label,.was-validated .custom-control-input:invalid~.custom-control-label{color:#dc3545}.custom-control-input.is-invalid~.custom-control-label:before,.was-validated .custom-control-input:invalid~.custom-control-label:before{border-color:#dc3545}.custom-control-input.is-invalid:checked~.custom-control-label:before,.was-validated .custom-control-input:invalid:checked~.custom-control-label:before{border-color:#e4606d;background-color:#e4606d}.custom-control-input.is-invalid:focus~.custom-control-label:before,.was-validated .custom-control-input:invalid:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.custom-control-input.is-invalid:focus:not(:checked)~.custom-control-label:before,.custom-file-input.is-invalid~.custom-file-label,.was-validated .custom-control-input:invalid:focus:not(:checked)~.custom-control-label:before,.was-validated .custom-file-input:invalid~.custom-file-label{border-color:#dc3545}.custom-file-input.is-invalid:focus~.custom-file-label,.was-validated .custom-file-input:invalid:focus~.custom-file-label{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.form-inline{display:flex;flex-flow:row wrap;align-items:center}.form-inline .form-check{width:100%}@media(min-width:616px){.form-inline label{justify-content:center}.form-inline .form-group,.form-inline label{display:flex;align-items:center;margin-bottom:0}.form-inline .form-group{flex:0 0 auto;flex-flow:row wrap}.form-inline .form-control{display:inline-block;width:auto;vertical-align:middle}.form-inline .form-control-plaintext{display:inline-block}.form-inline .custom-select,.form-inline .input-group{width:auto}.form-inline .form-check{display:flex;align-items:center;justify-content:center;width:auto;padding-left:0}.form-inline .form-check-input{position:relative;flex-shrink:0;margin-top:0;margin-right:.25rem;margin-left:0}.form-inline .custom-control{align-items:center;justify-content:center}.form-inline .custom-control-label{margin-bottom:0}}.btn{display:inline-block;font-family:inherit;font-weight:700;color:#212529;text-align:center;vertical-align:middle;cursor:pointer;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none;background-color:transparent;border:1px solid transparent;padding:12px 32px;font-size:.875rem;line-height:20px;border-radius:8px;transition:color .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.btn{transition:none}}.btn:hover{color:#212529;text-decoration:none}.btn.focus,.btn:focus{outline:0;box-shadow:none}.btn.disabled,.btn:disabled{opacity:.65}a.btn.disabled,fieldset:disabled a.btn{pointer-events:none}.btn-primary{color:#495057;background-color:#fc0;border-color:#fc0}.btn-primary.focus,.btn-primary:focus,.btn-primary:hover{color:#495057;background-color:#d9ad00;border-color:#cca300}.btn-primary.focus,.btn-primary:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-primary.disabled,.btn-primary:disabled{color:#495057;background-color:#fc0;border-color:#fc0}.btn-primary:not(:disabled):not(.disabled).active,.btn-primary:not(:disabled):not(.disabled):active,.show>.btn-primary.dropdown-toggle{color:#495057;background-color:#cca300;border-color:#bf9900}.btn-primary:not(:disabled):not(.disabled).active:focus,.btn-primary:not(:disabled):not(.disabled):active:focus,.show>.btn-primary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-secondary{color:#fff;background-color:#212529;border-color:#212529}.btn-secondary.focus,.btn-secondary:focus,.btn-secondary:hover{color:#fff;background-color:#101214;border-color:#0a0c0d}.btn-secondary.focus,.btn-secondary:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-secondary.disabled,.btn-secondary:disabled{color:#fff;background-color:#212529;border-color:#212529}.btn-secondary:not(:disabled):not(.disabled).active,.btn-secondary:not(:disabled):not(.disabled):active,.show>.btn-secondary.dropdown-toggle{color:#fff;background-color:#0a0c0d;border-color:#050506}.btn-secondary:not(:disabled):not(.disabled).active:focus,.btn-secondary:not(:disabled):not(.disabled):active:focus,.show>.btn-secondary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-success{color:#fff;background-color:#28a745;border-color:#28a745}.btn-success.focus,.btn-success:focus,.btn-success:hover{color:#fff;background-color:#218838;border-color:#1e7e34}.btn-success.focus,.btn-success:focus{box-shadow:0 0 0 0 rgba(72,180,97,.5)}.btn-success.disabled,.btn-success:disabled{color:#fff;background-color:#28a745;border-color:#28a745}.btn-success:not(:disabled):not(.disabled).active,.btn-success:not(:disabled):not(.disabled):active,.show>.btn-success.dropdown-toggle{color:#fff;background-color:#1e7e34;border-color:#1c7430}.btn-success:not(:disabled):not(.disabled).active:focus,.btn-success:not(:disabled):not(.disabled):active:focus,.show>.btn-success.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(72,180,97,.5)}.btn-info{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-info.focus,.btn-info:focus,.btn-info:hover{color:#fff;background-color:#138496;border-color:#117a8b}.btn-info.focus,.btn-info:focus{box-shadow:0 0 0 0 rgba(58,176,195,.5)}.btn-info.disabled,.btn-info:disabled{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-info:not(:disabled):not(.disabled).active,.btn-info:not(:disabled):not(.disabled):active,.show>.btn-info.dropdown-toggle{color:#fff;background-color:#117a8b;border-color:#10707f}.btn-info:not(:disabled):not(.disabled).active:focus,.btn-info:not(:disabled):not(.disabled):active:focus,.show>.btn-info.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(58,176,195,.5)}.btn-warning{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-warning.focus,.btn-warning:focus,.btn-warning:hover{color:#495057;background-color:#e0a800;border-color:#d39e00}.btn-warning.focus,.btn-warning:focus{box-shadow:0 0 0 0 rgba(228,176,19,.5)}.btn-warning.disabled,.btn-warning:disabled{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-warning:not(:disabled):not(.disabled).active,.btn-warning:not(:disabled):not(.disabled):active,.show>.btn-warning.dropdown-toggle{color:#495057;background-color:#d39e00;border-color:#c69500}.btn-warning:not(:disabled):not(.disabled).active:focus,.btn-warning:not(:disabled):not(.disabled):active:focus,.show>.btn-warning.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,176,19,.5)}.btn-danger{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-danger.focus,.btn-danger:focus,.btn-danger:hover{color:#fff;background-color:#c82333;border-color:#bd2130}.btn-danger.focus,.btn-danger:focus{box-shadow:0 0 0 0 rgba(225,83,97,.5)}.btn-danger.disabled,.btn-danger:disabled{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-danger:not(:disabled):not(.disabled).active,.btn-danger:not(:disabled):not(.disabled):active,.show>.btn-danger.dropdown-toggle{color:#fff;background-color:#bd2130;border-color:#b21f2d}.btn-danger:not(:disabled):not(.disabled).active:focus,.btn-danger:not(:disabled):not(.disabled):active:focus,.show>.btn-danger.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(225,83,97,.5)}.btn-light{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-light.focus,.btn-light:focus,.btn-light:hover{color:#495057;background-color:#d6e5ee;border-color:#cddfea}.btn-light.focus,.btn-light:focus{box-shadow:0 0 0 0 rgba(216,221,225,.5)}.btn-light.disabled,.btn-light:disabled{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-light:not(:disabled):not(.disabled).active,.btn-light:not(:disabled):not(.disabled):active,.show>.btn-light.dropdown-toggle{color:#495057;background-color:#cddfea;border-color:#c4d9e6}.btn-light:not(:disabled):not(.disabled).active:focus,.btn-light:not(:disabled):not(.disabled):active:focus,.show>.btn-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(216,221,225,.5)}.btn-dark{color:#fff;background-color:#495057;border-color:#495057}.btn-dark.focus,.btn-dark:focus,.btn-dark:hover{color:#fff;background-color:#383d42;border-color:#32373b}.btn-dark.focus,.btn-dark:focus{box-shadow:0 0 0 0 rgba(100,106,112,.5)}.btn-dark.disabled,.btn-dark:disabled{color:#fff;background-color:#495057;border-color:#495057}.btn-dark:not(:disabled):not(.disabled).active,.btn-dark:not(:disabled):not(.disabled):active,.show>.btn-dark.dropdown-toggle{color:#fff;background-color:#32373b;border-color:#2c3034}.btn-dark:not(:disabled):not(.disabled).active:focus,.btn-dark:not(:disabled):not(.disabled):active:focus,.show>.btn-dark.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(100,106,112,.5)}.btn-primary-light{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-primary-light.focus,.btn-primary-light:focus,.btn-primary-light:hover{color:#495057;background-color:#ffedca;border-color:#ffe9bd}.btn-primary-light.focus,.btn-primary-light:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-primary-light.disabled,.btn-primary-light:disabled{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-primary-light:not(:disabled):not(.disabled).active,.btn-primary-light:not(:disabled):not(.disabled):active,.show>.btn-primary-light.dropdown-toggle{color:#495057;background-color:#ffe9bd;border-color:#ffe5b0}.btn-primary-light:not(:disabled):not(.disabled).active:focus,.btn-primary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-primary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-secondary-light{color:#495057;background-color:#fff;border-color:#fff}.btn-secondary-light.focus,.btn-secondary-light:focus,.btn-secondary-light:hover{color:#495057;background-color:#ececec;border-color:#e6e6e6}.btn-secondary-light.focus,.btn-secondary-light:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-secondary-light.disabled,.btn-secondary-light:disabled{color:#495057;background-color:#fff;border-color:#fff}.btn-secondary-light:not(:disabled):not(.disabled).active,.btn-secondary-light:not(:disabled):not(.disabled):active,.show>.btn-secondary-light.dropdown-toggle{color:#495057;background-color:#e6e6e6;border-color:#dfdfdf}.btn-secondary-light:not(:disabled):not(.disabled).active:focus,.btn-secondary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-secondary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-tertiary{color:#fff;background-color:#257af4;border-color:#257af4}.btn-tertiary.focus,.btn-tertiary:focus,.btn-tertiary:hover{color:#fff;background-color:#0c66e7;border-color:#0b60db}.btn-tertiary.focus,.btn-tertiary:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-tertiary.disabled,.btn-tertiary:disabled{color:#fff;background-color:#257af4;border-color:#257af4}.btn-tertiary:not(:disabled):not(.disabled).active,.btn-tertiary:not(:disabled):not(.disabled):active,.show>.btn-tertiary.dropdown-toggle{color:#fff;background-color:#0b60db;border-color:#0a5bcf}.btn-tertiary:not(:disabled):not(.disabled).active:focus,.btn-tertiary:not(:disabled):not(.disabled):active:focus,.show>.btn-tertiary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-tertiary-light{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-tertiary-light.focus,.btn-tertiary-light:focus,.btn-tertiary-light:hover{color:#495057;background-color:#bedffd;border-color:#b2d8fc}.btn-tertiary-light.focus,.btn-tertiary-light:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-tertiary-light.disabled,.btn-tertiary-light:disabled{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-tertiary-light:not(:disabled):not(.disabled).active,.btn-tertiary-light:not(:disabled):not(.disabled):active,.show>.btn-tertiary-light.dropdown-toggle{color:#495057;background-color:#b2d8fc;border-color:#a5d2fc}.btn-tertiary-light:not(:disabled):not(.disabled).active:focus,.btn-tertiary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-tertiary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-white{color:#495057;background-color:#fff;border-color:#fff}.btn-white.focus,.btn-white:focus,.btn-white:hover{color:#495057;background-color:#ececec;border-color:#e6e6e6}.btn-white.focus,.btn-white:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-white.disabled,.btn-white:disabled{color:#495057;background-color:#fff;border-color:#fff}.btn-white:not(:disabled):not(.disabled).active,.btn-white:not(:disabled):not(.disabled):active,.show>.btn-white.dropdown-toggle{color:#495057;background-color:#e6e6e6;border-color:#dfdfdf}.btn-white:not(:disabled):not(.disabled).active:focus,.btn-white:not(:disabled):not(.disabled):active:focus,.show>.btn-white.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-black{color:#fff;background-color:#212529;border-color:#212529}.btn-black.focus,.btn-black:focus,.btn-black:hover{color:#fff;background-color:#101214;border-color:#0a0c0d}.btn-black.focus,.btn-black:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-black.disabled,.btn-black:disabled{color:#fff;background-color:#212529;border-color:#212529}.btn-black:not(:disabled):not(.disabled).active,.btn-black:not(:disabled):not(.disabled):active,.show>.btn-black.dropdown-toggle{color:#fff;background-color:#0a0c0d;border-color:#050506}.btn-black:not(:disabled):not(.disabled).active:focus,.btn-black:not(:disabled):not(.disabled):active:focus,.show>.btn-black.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-blue{color:#fff;background-color:#257af4;border-color:#257af4}.btn-blue.focus,.btn-blue:focus,.btn-blue:hover{color:#fff;background-color:#0c66e7;border-color:#0b60db}.btn-blue.focus,.btn-blue:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-blue.disabled,.btn-blue:disabled{color:#fff;background-color:#257af4;border-color:#257af4}.btn-blue:not(:disabled):not(.disabled).active,.btn-blue:not(:disabled):not(.disabled):active,.show>.btn-blue.dropdown-toggle{color:#fff;background-color:#0b60db;border-color:#0a5bcf}.btn-blue:not(:disabled):not(.disabled).active:focus,.btn-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-light-blue{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-light-blue.focus,.btn-light-blue:focus,.btn-light-blue:hover{color:#495057;background-color:#bedffd;border-color:#b2d8fc}.btn-light-blue.focus,.btn-light-blue:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-light-blue.disabled,.btn-light-blue:disabled{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-light-blue:not(:disabled):not(.disabled).active,.btn-light-blue:not(:disabled):not(.disabled):active,.show>.btn-light-blue.dropdown-toggle{color:#495057;background-color:#b2d8fc;border-color:#a5d2fc}.btn-light-blue:not(:disabled):not(.disabled).active:focus,.btn-light-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-light-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-yellow{color:#495057;background-color:#fc0;border-color:#fc0}.btn-yellow.focus,.btn-yellow:focus,.btn-yellow:hover{color:#495057;background-color:#d9ad00;border-color:#cca300}.btn-yellow.focus,.btn-yellow:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-yellow.disabled,.btn-yellow:disabled{color:#495057;background-color:#fc0;border-color:#fc0}.btn-yellow:not(:disabled):not(.disabled).active,.btn-yellow:not(:disabled):not(.disabled):active,.show>.btn-yellow.dropdown-toggle{color:#495057;background-color:#cca300;border-color:#bf9900}.btn-yellow:not(:disabled):not(.disabled).active:focus,.btn-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-light-yellow{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-light-yellow.focus,.btn-light-yellow:focus,.btn-light-yellow:hover{color:#495057;background-color:#ffedca;border-color:#ffe9bd}.btn-light-yellow.focus,.btn-light-yellow:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-light-yellow.disabled,.btn-light-yellow:disabled{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-light-yellow:not(:disabled):not(.disabled).active,.btn-light-yellow:not(:disabled):not(.disabled):active,.show>.btn-light-yellow.dropdown-toggle{color:#495057;background-color:#ffe9bd;border-color:#ffe5b0}.btn-light-yellow:not(:disabled):not(.disabled).active:focus,.btn-light-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-light-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-orange{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-orange.focus,.btn-orange:focus,.btn-orange:hover{color:#fff;background-color:#d97700;border-color:#cc7000}.btn-orange.focus,.btn-orange:focus{box-shadow:0 0 0 0 rgba(228,131,13,.5)}.btn-orange.disabled,.btn-orange:disabled{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-orange:not(:disabled):not(.disabled).active,.btn-orange:not(:disabled):not(.disabled):active,.show>.btn-orange.dropdown-toggle{color:#fff;background-color:#cc7000;border-color:#bf6900}.btn-orange:not(:disabled):not(.disabled).active:focus,.btn-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,131,13,.5)}.btn-light-orange{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-light-orange.focus,.btn-light-orange:focus,.btn-light-orange:hover{color:#495057;background-color:#ffd68f;border-color:#ffd182}.btn-light-orange.focus,.btn-light-orange:focus{box-shadow:0 0 0 0 rgba(228,206,167,.5)}.btn-light-orange.disabled,.btn-light-orange:disabled{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-light-orange:not(:disabled):not(.disabled).active,.btn-light-orange:not(:disabled):not(.disabled):active,.show>.btn-light-orange.dropdown-toggle{color:#495057;background-color:#ffd182;border-color:#ffcd75}.btn-light-orange:not(:disabled):not(.disabled).active:focus,.btn-light-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-light-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,206,167,.5)}.btn-red{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-red.focus,.btn-red:focus,.btn-red:hover{color:#fff;background-color:#ff1313;border-color:#ff0606}.btn-red.focus,.btn-red:focus{box-shadow:0 0 0 0 rgba(255,87,87,.5)}.btn-red.disabled,.btn-red:disabled{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-red:not(:disabled):not(.disabled).active,.btn-red:not(:disabled):not(.disabled):active,.show>.btn-red.dropdown-toggle{color:#fff;background-color:#ff0606;border-color:#f80000}.btn-red:not(:disabled):not(.disabled).active:focus,.btn-red:not(:disabled):not(.disabled):active:focus,.show>.btn-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,87,87,.5)}.btn-light-red{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-light-red.focus,.btn-light-red:focus,.btn-light-red:hover{color:#495057;background-color:#ffc2bb;border-color:#ffb6ae}.btn-light-red.focus,.btn-light-red:focus{box-shadow:0 0 0 0 rgba(228,206,204,.5)}.btn-light-red.disabled,.btn-light-red:disabled{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-light-red:not(:disabled):not(.disabled).active,.btn-light-red:not(:disabled):not(.disabled):active,.show>.btn-light-red.dropdown-toggle{color:#495057;background-color:#ffb6ae;border-color:#ffaba1}.btn-light-red:not(:disabled):not(.disabled).active:focus,.btn-light-red:not(:disabled):not(.disabled):active:focus,.show>.btn-light-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,206,204,.5)}.btn-medium{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-medium.focus,.btn-medium:focus,.btn-medium:hover{color:#495057;background-color:#c1c8ce;border-color:#b9c2c9}.btn-medium.focus,.btn-medium:focus{box-shadow:0 0 0 0 rgba(193,198,203,.5)}.btn-medium.disabled,.btn-medium:disabled{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-medium:not(:disabled):not(.disabled).active,.btn-medium:not(:disabled):not(.disabled):active,.show>.btn-medium.dropdown-toggle{color:#495057;background-color:#b9c2c9;border-color:#b2bcc3}.btn-medium:not(:disabled):not(.disabled).active:focus,.btn-medium:not(:disabled):not(.disabled):active:focus,.show>.btn-medium.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(193,198,203,.5)}.btn-outline-primary{color:#fc0;border-color:#fc0}.btn-outline-primary:hover{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-primary.focus,.btn-outline-primary:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-primary.disabled,.btn-outline-primary:disabled{color:#fc0;background-color:transparent}.btn-outline-primary:not(:disabled):not(.disabled).active,.btn-outline-primary:not(:disabled):not(.disabled):active,.show>.btn-outline-primary.dropdown-toggle{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-primary:not(:disabled):not(.disabled).active:focus,.btn-outline-primary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-primary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-secondary{color:#212529;border-color:#212529}.btn-outline-secondary:hover{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-secondary.focus,.btn-outline-secondary:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-secondary.disabled,.btn-outline-secondary:disabled{color:#212529;background-color:transparent}.btn-outline-secondary:not(:disabled):not(.disabled).active,.btn-outline-secondary:not(:disabled):not(.disabled):active,.show>.btn-outline-secondary.dropdown-toggle{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-secondary:not(:disabled):not(.disabled).active:focus,.btn-outline-secondary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-secondary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-success{color:#28a745;border-color:#28a745}.btn-outline-success:hover{color:#fff;background-color:#28a745;border-color:#28a745}.btn-outline-success.focus,.btn-outline-success:focus{box-shadow:0 0 0 0 rgba(40,167,69,.5)}.btn-outline-success.disabled,.btn-outline-success:disabled{color:#28a745;background-color:transparent}.btn-outline-success:not(:disabled):not(.disabled).active,.btn-outline-success:not(:disabled):not(.disabled):active,.show>.btn-outline-success.dropdown-toggle{color:#fff;background-color:#28a745;border-color:#28a745}.btn-outline-success:not(:disabled):not(.disabled).active:focus,.btn-outline-success:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-success.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(40,167,69,.5)}.btn-outline-info{color:#17a2b8;border-color:#17a2b8}.btn-outline-info:hover{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-outline-info.focus,.btn-outline-info:focus{box-shadow:0 0 0 0 rgba(23,162,184,.5)}.btn-outline-info.disabled,.btn-outline-info:disabled{color:#17a2b8;background-color:transparent}.btn-outline-info:not(:disabled):not(.disabled).active,.btn-outline-info:not(:disabled):not(.disabled):active,.show>.btn-outline-info.dropdown-toggle{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-outline-info:not(:disabled):not(.disabled).active:focus,.btn-outline-info:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-info.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(23,162,184,.5)}.btn-outline-warning{color:#ffc107;border-color:#ffc107}.btn-outline-warning:hover{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-outline-warning.focus,.btn-outline-warning:focus{box-shadow:0 0 0 0 rgba(255,193,7,.5)}.btn-outline-warning.disabled,.btn-outline-warning:disabled{color:#ffc107;background-color:transparent}.btn-outline-warning:not(:disabled):not(.disabled).active,.btn-outline-warning:not(:disabled):not(.disabled):active,.show>.btn-outline-warning.dropdown-toggle{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-outline-warning:not(:disabled):not(.disabled).active:focus,.btn-outline-warning:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-warning.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,193,7,.5)}.btn-outline-danger{color:#dc3545;border-color:#dc3545}.btn-outline-danger:hover{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-outline-danger.focus,.btn-outline-danger:focus{box-shadow:0 0 0 0 rgba(220,53,69,.5)}.btn-outline-danger.disabled,.btn-outline-danger:disabled{color:#dc3545;background-color:transparent}.btn-outline-danger:not(:disabled):not(.disabled).active,.btn-outline-danger:not(:disabled):not(.disabled):active,.show>.btn-outline-danger.dropdown-toggle{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-outline-danger:not(:disabled):not(.disabled).active:focus,.btn-outline-danger:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-danger.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(220,53,69,.5)}.btn-outline-light{color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light:hover{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light.focus,.btn-outline-light:focus{box-shadow:0 0 0 0 rgba(241,246,249,.5)}.btn-outline-light.disabled,.btn-outline-light:disabled{color:#f1f6f9;background-color:transparent}.btn-outline-light:not(:disabled):not(.disabled).active,.btn-outline-light:not(:disabled):not(.disabled):active,.show>.btn-outline-light.dropdown-toggle{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light:not(:disabled):not(.disabled).active:focus,.btn-outline-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(241,246,249,.5)}.btn-outline-dark{color:#495057;border-color:#495057}.btn-outline-dark:hover{color:#fff;background-color:#495057;border-color:#495057}.btn-outline-dark.focus,.btn-outline-dark:focus{box-shadow:0 0 0 0 rgba(73,80,87,.5)}.btn-outline-dark.disabled,.btn-outline-dark:disabled{color:#495057;background-color:transparent}.btn-outline-dark:not(:disabled):not(.disabled).active,.btn-outline-dark:not(:disabled):not(.disabled):active,.show>.btn-outline-dark.dropdown-toggle{color:#fff;background-color:#495057;border-color:#495057}.btn-outline-dark:not(:disabled):not(.disabled).active:focus,.btn-outline-dark:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-dark.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(73,80,87,.5)}.btn-outline-primary-light{color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light:hover{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light.focus,.btn-outline-primary-light:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-primary-light.disabled,.btn-outline-primary-light:disabled{color:#fffaf0;background-color:transparent}.btn-outline-primary-light:not(:disabled):not(.disabled).active,.btn-outline-primary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-primary-light.dropdown-toggle{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-primary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-primary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-secondary-light{color:#fff;border-color:#fff}.btn-outline-secondary-light:hover{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-secondary-light.focus,.btn-outline-secondary-light:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-secondary-light.disabled,.btn-outline-secondary-light:disabled{color:#fff;background-color:transparent}.btn-outline-secondary-light:not(:disabled):not(.disabled).active,.btn-outline-secondary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-secondary-light.dropdown-toggle{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-secondary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-secondary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-secondary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-tertiary{color:#257af4;border-color:#257af4}.btn-outline-tertiary:hover{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-tertiary.focus,.btn-outline-tertiary:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-tertiary.disabled,.btn-outline-tertiary:disabled{color:#257af4;background-color:transparent}.btn-outline-tertiary:not(:disabled):not(.disabled).active,.btn-outline-tertiary:not(:disabled):not(.disabled):active,.show>.btn-outline-tertiary.dropdown-toggle{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-tertiary:not(:disabled):not(.disabled).active:focus,.btn-outline-tertiary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-tertiary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-tertiary-light{color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light:hover{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light.focus,.btn-outline-tertiary-light:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-tertiary-light.disabled,.btn-outline-tertiary-light:disabled{color:#e3f1fe;background-color:transparent}.btn-outline-tertiary-light:not(:disabled):not(.disabled).active,.btn-outline-tertiary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-tertiary-light.dropdown-toggle{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-tertiary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-tertiary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-white{color:#fff;border-color:#fff}.btn-outline-white:hover{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-white.focus,.btn-outline-white:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-white.disabled,.btn-outline-white:disabled{color:#fff;background-color:transparent}.btn-outline-white:not(:disabled):not(.disabled).active,.btn-outline-white:not(:disabled):not(.disabled):active,.show>.btn-outline-white.dropdown-toggle{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-white:not(:disabled):not(.disabled).active:focus,.btn-outline-white:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-white.dropdown-toggle:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-black{color:#212529;border-color:#212529}.btn-outline-black:hover{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-black.focus,.btn-outline-black:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-black.disabled,.btn-outline-black:disabled{color:#212529;background-color:transparent}.btn-outline-black:not(:disabled):not(.disabled).active,.btn-outline-black:not(:disabled):not(.disabled):active,.show>.btn-outline-black.dropdown-toggle{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-black:not(:disabled):not(.disabled).active:focus,.btn-outline-black:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-black.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-blue{color:#257af4;border-color:#257af4}.btn-outline-blue:hover{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-blue.focus,.btn-outline-blue:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-blue.disabled,.btn-outline-blue:disabled{color:#257af4;background-color:transparent}.btn-outline-blue:not(:disabled):not(.disabled).active,.btn-outline-blue:not(:disabled):not(.disabled):active,.show>.btn-outline-blue.dropdown-toggle{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-blue:not(:disabled):not(.disabled).active:focus,.btn-outline-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-light-blue{color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue:hover{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue.focus,.btn-outline-light-blue:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-light-blue.disabled,.btn-outline-light-blue:disabled{color:#e3f1fe;background-color:transparent}.btn-outline-light-blue:not(:disabled):not(.disabled).active,.btn-outline-light-blue:not(:disabled):not(.disabled):active,.show>.btn-outline-light-blue.dropdown-toggle{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue:not(:disabled):not(.disabled).active:focus,.btn-outline-light-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-yellow{color:#fc0;border-color:#fc0}.btn-outline-yellow:hover{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-yellow.focus,.btn-outline-yellow:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-yellow.disabled,.btn-outline-yellow:disabled{color:#fc0;background-color:transparent}.btn-outline-yellow:not(:disabled):not(.disabled).active,.btn-outline-yellow:not(:disabled):not(.disabled):active,.show>.btn-outline-yellow.dropdown-toggle{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-yellow:not(:disabled):not(.disabled).active:focus,.btn-outline-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-light-yellow{color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow:hover{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow.focus,.btn-outline-light-yellow:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-light-yellow.disabled,.btn-outline-light-yellow:disabled{color:#fffaf0;background-color:transparent}.btn-outline-light-yellow:not(:disabled):not(.disabled).active,.btn-outline-light-yellow:not(:disabled):not(.disabled):active,.show>.btn-outline-light-yellow.dropdown-toggle{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow:not(:disabled):not(.disabled).active:focus,.btn-outline-light-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-orange{color:#ff8c00;border-color:#ff8c00}.btn-outline-orange:hover{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-outline-orange.focus,.btn-outline-orange:focus{box-shadow:0 0 0 0 rgba(255,140,0,.5)}.btn-outline-orange.disabled,.btn-outline-orange:disabled{color:#ff8c00;background-color:transparent}.btn-outline-orange:not(:disabled):not(.disabled).active,.btn-outline-orange:not(:disabled):not(.disabled):active,.show>.btn-outline-orange.dropdown-toggle{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-outline-orange:not(:disabled):not(.disabled).active:focus,.btn-outline-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,140,0,.5)}.btn-outline-light-orange{color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange:hover{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange.focus,.btn-outline-light-orange:focus{box-shadow:0 0 0 0 rgba(255,228,181,.5)}.btn-outline-light-orange.disabled,.btn-outline-light-orange:disabled{color:#ffe4b5;background-color:transparent}.btn-outline-light-orange:not(:disabled):not(.disabled).active,.btn-outline-light-orange:not(:disabled):not(.disabled):active,.show>.btn-outline-light-orange.dropdown-toggle{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange:not(:disabled):not(.disabled).active:focus,.btn-outline-light-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,228,181,.5)}.btn-outline-red{color:#ff3939;border-color:#ff3939}.btn-outline-red:hover{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-outline-red.focus,.btn-outline-red:focus{box-shadow:0 0 0 0 rgba(255,57,57,.5)}.btn-outline-red.disabled,.btn-outline-red:disabled{color:#ff3939;background-color:transparent}.btn-outline-red:not(:disabled):not(.disabled).active,.btn-outline-red:not(:disabled):not(.disabled):active,.show>.btn-outline-red.dropdown-toggle{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-outline-red:not(:disabled):not(.disabled).active:focus,.btn-outline-red:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,57,57,.5)}.btn-outline-light-red{color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red:hover{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red.focus,.btn-outline-light-red:focus{box-shadow:0 0 0 0 rgba(255,228,225,.5)}.btn-outline-light-red.disabled,.btn-outline-light-red:disabled{color:#ffe4e1;background-color:transparent}.btn-outline-light-red:not(:disabled):not(.disabled).active,.btn-outline-light-red:not(:disabled):not(.disabled):active,.show>.btn-outline-light-red.dropdown-toggle{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red:not(:disabled):not(.disabled).active:focus,.btn-outline-light-red:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,228,225,.5)}.btn-outline-medium{color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium:hover{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium.focus,.btn-outline-medium:focus{box-shadow:0 0 0 0 rgba(214,219,223,.5)}.btn-outline-medium.disabled,.btn-outline-medium:disabled{color:#d6dbdf;background-color:transparent}.btn-outline-medium:not(:disabled):not(.disabled).active,.btn-outline-medium:not(:disabled):not(.disabled):active,.show>.btn-outline-medium.dropdown-toggle{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium:not(:disabled):not(.disabled).active:focus,.btn-outline-medium:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-medium.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(214,219,223,.5)}.btn-link{font-weight:400;color:#ff8c00;text-decoration:none}.btn-link:hover{color:#ff8c00;text-decoration:underline}.btn-link.focus,.btn-link:focus{text-decoration:underline;box-shadow:none}.btn-link.disabled,.btn-link:disabled{color:#d6dbdf;pointer-events:none}.btn-group-lg>.btn,.btn-lg{padding:16px 32px;font-size:1.125rem;line-height:26px;border-radius:8px}.btn-group-sm>.btn,.btn-sm{padding:12px 32px;font-size:.875rem;line-height:20px;border-radius:8px}.btn-block{display:block;width:100%}.btn-block+.btn-block{margin-top:24px}input[type=button].btn-block,input[type=reset].btn-block,input[type=submit].btn-block{width:100%}.fade{transition:opacity .15s linear}@media(prefers-reduced-motion:reduce){.fade{transition:none}}.fade:not(.show){opacity:0}.collapse:not(.show){display:none}.collapsing{position:relative;height:0;overflow:hidden;transition:height .35s ease}@media(prefers-reduced-motion:reduce){.collapsing{transition:none}}.dropdown,.dropleft,.dropright,.dropup{position:relative}.dropdown-toggle{white-space:nowrap}.dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:.3em solid;border-right:.3em solid transparent;border-bottom:0;border-left:.3em solid transparent}.dropdown-toggle:empty:after{margin-left:0}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:10rem;padding:.5rem 0;margin:.125rem 0 0;font-size:1rem;color:#212529;text-align:left;list-style:none;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.15);border-radius:8px}.dropdown-menu-left{right:auto;left:0}.dropdown-menu-right{right:0;left:auto}@media(min-width:400px){.dropdown-menu-xs-left{right:auto;left:0}.dropdown-menu-xs-right{right:0;left:auto}}@media(min-width:616px){.dropdown-menu-sm-left{right:auto;left:0}.dropdown-menu-sm-right{right:0;left:auto}}@media(min-width:768px){.dropdown-menu-md-left{right:auto;left:0}.dropdown-menu-md-right{right:0;left:auto}}@media(min-width:980px){.dropdown-menu-lg-left{right:auto;left:0}.dropdown-menu-lg-right{right:0;left:auto}}@media(min-width:1240px){.dropdown-menu-xl-left{right:auto;left:0}.dropdown-menu-xl-right{right:0;left:auto}}.dropup .dropdown-menu{top:auto;bottom:100%;margin-top:0;margin-bottom:.125rem}.dropup .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:0;border-right:.3em solid transparent;border-bottom:.3em solid;border-left:.3em solid transparent}.dropup .dropdown-toggle:empty:after{margin-left:0}.dropright .dropdown-menu{top:0;right:auto;left:100%;margin-top:0;margin-left:.125rem}.dropright .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:.3em solid transparent;border-right:0;border-bottom:.3em solid transparent;border-left:.3em solid}.dropright .dropdown-toggle:empty:after{margin-left:0}.dropright .dropdown-toggle:after{vertical-align:0}.dropleft .dropdown-menu{top:0;right:100%;left:auto;margin-top:0;margin-right:.125rem}.dropleft .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";display:none}.dropleft .dropdown-toggle:before{display:inline-block;margin-right:.255em;vertical-align:.255em;content:"";border-top:.3em solid transparent;border-right:.3em solid;border-bottom:.3em solid transparent}.dropleft .dropdown-toggle:empty:after{margin-left:0}.dropleft .dropdown-toggle:before{vertical-align:0}.dropdown-menu[x-placement^=bottom],.dropdown-menu[x-placement^=left],.dropdown-menu[x-placement^=right],.dropdown-menu[x-placement^=top]{right:auto;bottom:auto}.dropdown-divider{height:0;margin:4px 0;overflow:hidden;border-top:1px solid #e9ecef}.dropdown-item{display:block;width:100%;padding:.25rem 1.5rem;clear:both;font-weight:400;color:#495057;text-align:inherit;white-space:nowrap;background-color:transparent;border:0}.dropdown-item:focus,.dropdown-item:hover{color:#3d4349;text-decoration:none;background-color:#f1f6f9}.dropdown-item.active,.dropdown-item:active{color:#fff;text-decoration:none;background-color:#fc0}.dropdown-item.disabled,.dropdown-item:disabled{color:#6c757d;pointer-events:none;background-color:transparent}.dropdown-menu.show{display:block}.dropdown-header{display:block;padding:.5rem 1.5rem;margin-bottom:0;font-size:.875rem;color:#6c757d;white-space:nowrap}.dropdown-item-text{display:block;padding:.25rem 1.5rem;color:#495057}.btn-group,.btn-group-vertical{position:relative;display:inline-flex;vertical-align:middle}.btn-group-vertical>.btn,.btn-group>.btn{position:relative;flex:1 1 auto}.btn-group-vertical>.btn.active,.btn-group-vertical>.btn:active,.btn-group-vertical>.btn:focus,.btn-group-vertical>.btn:hover,.btn-group>.btn.active,.btn-group>.btn:active,.btn-group>.btn:focus,.btn-group>.btn:hover{z-index:1}.btn-toolbar{display:flex;flex-wrap:wrap;justify-content:flex-start}.btn-toolbar .input-group{width:auto}.btn-group>.btn-group:not(:first-child),.btn-group>.btn:not(:first-child){margin-left:-1px}.btn-group>.btn-group:not(:last-child)>.btn,.btn-group>.btn:not(:last-child):not(.dropdown-toggle){border-top-right-radius:0;border-bottom-right-radius:0}.btn-group>.btn-group:not(:first-child)>.btn,.btn-group>.btn:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.dropdown-toggle-split{padding-right:24px;padding-left:24px}.dropdown-toggle-split:after,.dropright .dropdown-toggle-split:after,.dropup .dropdown-toggle-split:after{margin-left:0}.dropleft .dropdown-toggle-split:before{margin-right:0}.btn-group-lg>.btn+.dropdown-toggle-split,.btn-group-sm>.btn+.dropdown-toggle-split,.btn-lg+.dropdown-toggle-split,.btn-sm+.dropdown-toggle-split{padding-right:24px;padding-left:24px}.btn-group-vertical{flex-direction:column;align-items:flex-start;justify-content:center}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group{width:100%}.btn-group-vertical>.btn-group:not(:first-child),.btn-group-vertical>.btn:not(:first-child){margin-top:-1px}.btn-group-vertical>.btn-group:not(:last-child)>.btn,.btn-group-vertical>.btn:not(:last-child):not(.dropdown-toggle){border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn-group:not(:first-child)>.btn,.btn-group-vertical>.btn:not(:first-child){border-top-left-radius:0;border-top-right-radius:0}.btn-group-toggle>.btn,.btn-group-toggle>.btn-group>.btn{margin-bottom:0}.btn-group-toggle>.btn-group>.btn input[type=checkbox],.btn-group-toggle>.btn-group>.btn input[type=radio],.btn-group-toggle>.btn input[type=checkbox],.btn-group-toggle>.btn input[type=radio]{position:absolute;clip:rect(0,0,0,0);pointer-events:none}.input-group{position:relative;display:flex;flex-wrap:wrap;align-items:stretch;width:100%}.input-group>.custom-file,.input-group>.custom-select,.input-group>.form-control,.input-group>.form-control-plaintext{position:relative;flex:1 1 0%;min-width:0;margin-bottom:0}.input-group>.custom-file+.custom-file,.input-group>.custom-file+.custom-select,.input-group>.custom-file+.form-control,.input-group>.custom-select+.custom-file,.input-group>.custom-select+.custom-select,.input-group>.custom-select+.form-control,.input-group>.form-control+.custom-file,.input-group>.form-control+.custom-select,.input-group>.form-control+.form-control,.input-group>.form-control-plaintext+.custom-file,.input-group>.form-control-plaintext+.custom-select,.input-group>.form-control-plaintext+.form-control{margin-left:-1px}.input-group>.custom-file .custom-file-input:focus~.custom-file-label,.input-group>.custom-select:focus,.input-group>.form-control:focus{z-index:3}.input-group>.custom-file .custom-file-input:focus{z-index:4}.input-group>.custom-select:not(:last-child),.input-group>.form-control:not(:last-child){border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.custom-select:not(:first-child),.input-group>.form-control:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.input-group>.custom-file{display:flex;align-items:center}.input-group>.custom-file:not(:last-child) .custom-file-label,.input-group>.custom-file:not(:last-child) .custom-file-label:after{border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.custom-file:not(:first-child) .custom-file-label{border-top-left-radius:0;border-bottom-left-radius:0}.input-group-append,.input-group-prepend{display:flex}.input-group-append .btn,.input-group-prepend .btn{position:relative;z-index:2}.input-group-append .btn:focus,.input-group-prepend .btn:focus{z-index:3}.input-group-append .btn+.btn,.input-group-append .btn+.input-group-text,.input-group-append .input-group-text+.btn,.input-group-append .input-group-text+.input-group-text,.input-group-prepend .btn+.btn,.input-group-prepend .btn+.input-group-text,.input-group-prepend .input-group-text+.btn,.input-group-prepend .input-group-text+.input-group-text{margin-left:-1px}.input-group-prepend{margin-right:-1px}.input-group-append{margin-left:-1px}.input-group-text{display:flex;align-items:center;padding:.375rem .75rem;margin-bottom:0;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;text-align:center;white-space:nowrap;background-color:#e9ecef;border:1px solid #ced4da;border-radius:8px}.input-group-text input[type=checkbox],.input-group-text input[type=radio]{margin-top:0}.input-group-lg>.custom-select,.input-group-lg>.form-control:not(textarea){height:calc(1.5em + 1rem + 2px)}.input-group-lg>.custom-select,.input-group-lg>.form-control,.input-group-lg>.input-group-append>.btn,.input-group-lg>.input-group-append>.input-group-text,.input-group-lg>.input-group-prepend>.btn,.input-group-lg>.input-group-prepend>.input-group-text{padding:.5rem 1rem;font-size:1.125rem;line-height:1.5;border-radius:8px}.input-group-sm>.custom-select,.input-group-sm>.form-control:not(textarea){height:calc(1.5em + .5rem + 2px)}.input-group-sm>.custom-select,.input-group-sm>.form-control,.input-group-sm>.input-group-append>.btn,.input-group-sm>.input-group-append>.input-group-text,.input-group-sm>.input-group-prepend>.btn,.input-group-sm>.input-group-prepend>.input-group-text{padding:.25rem .5rem;font-size:.875rem;line-height:1.5;border-radius:8px}.input-group-lg>.custom-select,.input-group-sm>.custom-select{padding-right:1.75rem}.input-group>.input-group-append:last-child>.btn:not(:last-child):not(.dropdown-toggle),.input-group>.input-group-append:last-child>.input-group-text:not(:last-child),.input-group>.input-group-append:not(:last-child)>.btn,.input-group>.input-group-append:not(:last-child)>.input-group-text,.input-group>.input-group-prepend>.btn,.input-group>.input-group-prepend>.input-group-text{border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.input-group-append>.btn,.input-group>.input-group-append>.input-group-text,.input-group>.input-group-prepend:first-child>.btn:not(:first-child),.input-group>.input-group-prepend:first-child>.input-group-text:not(:first-child),.input-group>.input-group-prepend:not(:first-child)>.btn,.input-group>.input-group-prepend:not(:first-child)>.input-group-text{border-top-left-radius:0;border-bottom-left-radius:0}.custom-control{position:relative;display:block;min-height:1.5rem;padding-left:1.5rem}.custom-control-inline{display:inline-flex;margin-right:1rem}.custom-control-input{position:absolute;left:0;z-index:-1;width:1rem;height:1.25rem;opacity:0}.custom-control-input:checked~.custom-control-label:before{color:#fff;border-color:#fc0;background-color:#fc0}.custom-control-input:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-control-input:focus:not(:checked)~.custom-control-label:before{border-color:#ffe680}.custom-control-input:not(:disabled):active~.custom-control-label:before{color:#fff;background-color:#fff0b3;border-color:#fff0b3}.custom-control-input:disabled~.custom-control-label,.custom-control-input[disabled]~.custom-control-label{color:#6c757d}.custom-control-input:disabled~.custom-control-label:before,.custom-control-input[disabled]~.custom-control-label:before{background-color:#e9ecef}.custom-control-label{position:relative;margin-bottom:0;vertical-align:top}.custom-control-label:before{pointer-events:none;background-color:#fff;border:1px solid #d6dbdf}.custom-control-label:after,.custom-control-label:before{position:absolute;top:.25rem;left:-1.5rem;display:block;width:1rem;height:1rem;content:""}.custom-control-label:after{background:no-repeat 50%/50% 50%}.custom-checkbox .custom-control-label:before{border-radius:8px}.custom-checkbox .custom-control-input:checked~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%23fff' d='M6.564.75l-3.59 3.612-1.538-1.55L0 4.26l2.974 2.99L8 2.193z'/%3E%3C/svg%3E")}.custom-checkbox .custom-control-input:indeterminate~.custom-control-label:before{border-color:#fc0;background-color:#fc0}.custom-checkbox .custom-control-input:indeterminate~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='4'%3E%3Cpath stroke='%23fff' d='M0 2h4'/%3E%3C/svg%3E")}.custom-checkbox .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-checkbox .custom-control-input:disabled:indeterminate~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-radio .custom-control-label:before{border-radius:50%}.custom-radio .custom-control-input:checked~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' viewBox='-4 -4 8 8'%3E%3Ccircle r='3' fill='%23fff'/%3E%3C/svg%3E")}.custom-radio .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-switch{padding-left:2.25rem}.custom-switch .custom-control-label:before{left:-2.25rem;width:1.75rem;pointer-events:all;border-radius:.5rem}.custom-switch .custom-control-label:after{top:calc(.25rem + 2px);left:calc(-2.25rem + 2px);width:calc(1rem - 4px);height:calc(1rem - 4px);background-color:#d6dbdf;border-radius:.5rem;transition:transform .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.custom-switch .custom-control-label:after{transition:none}}.custom-switch .custom-control-input:checked~.custom-control-label:after{background-color:#fff;transform:translateX(.75rem)}.custom-switch .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-select{display:inline-block;width:100%;height:calc(1.5em + .75rem + 2px);padding:.375rem 1.75rem .375rem .75rem;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;vertical-align:middle;background:#fff url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px;border:1px solid #ced4da;border-radius:8px;-webkit-appearance:none;-moz-appearance:none;appearance:none}.custom-select:focus{border-color:#ffe680;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-select:focus::-ms-value{color:#6c757d;background-color:#fff}.custom-select[multiple],.custom-select[size]:not([size="1"]){height:auto;padding-right:.75rem;background-image:none}.custom-select:disabled{color:#6c757d;background-color:#e9ecef}.custom-select::-ms-expand{display:none}.custom-select:-moz-focusring{color:transparent;text-shadow:0 0 0 #6c757d}.custom-select-sm{height:calc(1.5em + .5rem + 2px);padding-top:.25rem;padding-bottom:.25rem;padding-left:.5rem;font-size:.875rem}.custom-select-lg{height:calc(1.5em + 1rem + 2px);padding-top:.5rem;padding-bottom:.5rem;padding-left:1rem;font-size:1.125rem}.custom-file{display:inline-block;margin-bottom:0}.custom-file,.custom-file-input{position:relative;width:100%;height:calc(1.5em + .75rem + 2px)}.custom-file-input{z-index:2;margin:0;opacity:0}.custom-file-input:focus~.custom-file-label{border-color:#ffe680;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-file-input:disabled~.custom-file-label,.custom-file-input[disabled]~.custom-file-label{background-color:#e9ecef}.custom-file-input:lang(en)~.custom-file-label:after{content:"Browse"}.custom-file-input~.custom-file-label[data-browse]:after{content:attr(data-browse)}.custom-file-label{left:0;z-index:1;height:calc(1.5em + .75rem + 2px);font-weight:400;background-color:#fff;border:1px solid #ced4da;border-radius:8px}.custom-file-label,.custom-file-label:after{position:absolute;top:0;right:0;padding:.375rem .75rem;line-height:1.5;color:#6c757d}.custom-file-label:after{bottom:0;z-index:3;display:block;height:calc(1.5em + .75rem);content:"Browse";background-color:#e9ecef;border-left:inherit;border-radius:0 8px 8px 0}.custom-range{width:100%;height:1.4rem;padding:0;background-color:transparent;-webkit-appearance:none;-moz-appearance:none;appearance:none}.custom-range:focus{outline:none}.custom-range:focus::-webkit-slider-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range:focus::-moz-range-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range:focus::-ms-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range::-moz-focus-outer{border:0}.custom-range::-webkit-slider-thumb{width:1rem;height:1rem;margin-top:-.25rem;background-color:#fc0;border:0;border-radius:1rem;-webkit-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;-webkit-appearance:none;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-webkit-slider-thumb{-webkit-transition:none;transition:none}}.custom-range::-webkit-slider-thumb:active{background-color:#fff0b3}.custom-range::-webkit-slider-runnable-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:#dee2e6;border-color:transparent;border-radius:1rem}.custom-range::-moz-range-thumb{width:1rem;height:1rem;background-color:#fc0;border:0;border-radius:1rem;-moz-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;-moz-appearance:none;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-moz-range-thumb{-moz-transition:none;transition:none}}.custom-range::-moz-range-thumb:active{background-color:#fff0b3}.custom-range::-moz-range-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:#dee2e6;border-color:transparent;border-radius:1rem}.custom-range::-ms-thumb{width:1rem;height:1rem;margin-top:0;margin-right:.2rem;margin-left:.2rem;background-color:#fc0;border:0;border-radius:1rem;-ms-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-ms-thumb{-ms-transition:none;transition:none}}.custom-range::-ms-thumb:active{background-color:#fff0b3}.custom-range::-ms-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:transparent;border-color:transparent;border-width:.5rem}.custom-range::-ms-fill-lower,.custom-range::-ms-fill-upper{background-color:#dee2e6;border-radius:1rem}.custom-range::-ms-fill-upper{margin-right:15px}.custom-range:disabled::-webkit-slider-thumb{background-color:#d6dbdf}.custom-range:disabled::-webkit-slider-runnable-track{cursor:default}.custom-range:disabled::-moz-range-thumb{background-color:#d6dbdf}.custom-range:disabled::-moz-range-track{cursor:default}.custom-range:disabled::-ms-thumb{background-color:#d6dbdf}.custom-control-label:before,.custom-file-label,.custom-select{transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.custom-control-label:before,.custom-file-label,.custom-select{transition:none}}.nav{display:flex;flex-wrap:wrap;padding-left:0;margin-bottom:0;list-style:none}.nav-link{display:block;padding:0}.nav-link:focus,.nav-link:hover{text-decoration:none}.nav-link.disabled{color:#d6dbdf;pointer-events:none;cursor:default}.nav-tabs{border-bottom:1px solid #6c757d}.nav-tabs .nav-item{margin-bottom:-1px}.nav-tabs .nav-link{border:1px solid transparent;border-top-left-radius:8px;border-top-right-radius:8px}.nav-tabs .nav-link:focus,.nav-tabs .nav-link:hover{border-color:transparent}.nav-tabs .nav-link.disabled{color:#d6dbdf;background-color:transparent;border-color:transparent}.nav-tabs .nav-item.show .nav-link,.nav-tabs .nav-link.active{color:#257af4;background-color:#fff;border-color:#6c757d}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-left-radius:0;border-top-right-radius:0}.nav-pills .nav-link{border-radius:8px}.nav-pills .nav-link.active,.nav-pills .show>.nav-link{color:#fff;background-color:#fc0}.nav-fill .nav-item{flex:1 1 auto;text-align:center}.nav-justified .nav-item{flex-basis:0;flex-grow:1;text-align:center}.tab-content>.tab-pane{display:none}.tab-content>.active{display:block}.navbar{position:relative;padding:24px 0}.navbar,.navbar .container,.navbar .container-fluid,.navbar .container-lg,.navbar .container-md,.navbar .container-sm,.navbar .container-xl,.navbar .container-xs{display:flex;flex-wrap:wrap;align-items:center;justify-content:space-between}.navbar-brand{display:inline-block;padding-top:-.09375rem;padding-bottom:-.09375rem;margin-right:0;font-size:1.125rem;line-height:inherit;white-space:nowrap}.navbar-brand:focus,.navbar-brand:hover{text-decoration:none}.navbar-nav{display:flex;flex-direction:column;padding-left:0;margin-bottom:0;list-style:none}.navbar-nav .nav-link{padding-right:0;padding-left:0}.navbar-nav .dropdown-menu{position:static;float:none}.navbar-text{display:inline-block;padding-top:0;padding-bottom:0}.navbar-collapse{flex-basis:100%;flex-grow:1;align-items:center}.navbar-toggler{padding:.25rem .75rem;font-size:1.125rem;line-height:1;background-color:transparent;border:1px solid transparent;border-radius:8px}.navbar-toggler:focus,.navbar-toggler:hover{text-decoration:none}.navbar-toggler-icon{display:inline-block;width:1.5em;height:1.5em;vertical-align:middle;content:"";background:no-repeat 50%;background-size:100% 100%}@media(max-width:399.98px){.navbar-expand-xs>.container,.navbar-expand-xs>.container-fluid,.navbar-expand-xs>.container-lg,.navbar-expand-xs>.container-md,.navbar-expand-xs>.container-sm,.navbar-expand-xs>.container-xl,.navbar-expand-xs>.container-xs{padding-right:0;padding-left:0}}@media(min-width:400px){.navbar-expand-xs{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-xs .navbar-nav{flex-direction:row}.navbar-expand-xs .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-xs .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-xs>.container,.navbar-expand-xs>.container-fluid,.navbar-expand-xs>.container-lg,.navbar-expand-xs>.container-md,.navbar-expand-xs>.container-sm,.navbar-expand-xs>.container-xl,.navbar-expand-xs>.container-xs{flex-wrap:nowrap}.navbar-expand-xs .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-xs .navbar-toggler{display:none}}@media(max-width:615.98px){.navbar-expand-sm>.container,.navbar-expand-sm>.container-fluid,.navbar-expand-sm>.container-lg,.navbar-expand-sm>.container-md,.navbar-expand-sm>.container-sm,.navbar-expand-sm>.container-xl,.navbar-expand-sm>.container-xs{padding-right:0;padding-left:0}}@media(min-width:616px){.navbar-expand-sm{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-sm .navbar-nav{flex-direction:row}.navbar-expand-sm .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-sm .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-sm>.container,.navbar-expand-sm>.container-fluid,.navbar-expand-sm>.container-lg,.navbar-expand-sm>.container-md,.navbar-expand-sm>.container-sm,.navbar-expand-sm>.container-xl,.navbar-expand-sm>.container-xs{flex-wrap:nowrap}.navbar-expand-sm .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-sm .navbar-toggler{display:none}}@media(max-width:767.98px){.navbar-expand-md>.container,.navbar-expand-md>.container-fluid,.navbar-expand-md>.container-lg,.navbar-expand-md>.container-md,.navbar-expand-md>.container-sm,.navbar-expand-md>.container-xl,.navbar-expand-md>.container-xs{padding-right:0;padding-left:0}}@media(min-width:768px){.navbar-expand-md{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-md .navbar-nav{flex-direction:row}.navbar-expand-md .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-md .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-md>.container,.navbar-expand-md>.container-fluid,.navbar-expand-md>.container-lg,.navbar-expand-md>.container-md,.navbar-expand-md>.container-sm,.navbar-expand-md>.container-xl,.navbar-expand-md>.container-xs{flex-wrap:nowrap}.navbar-expand-md .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-md .navbar-toggler{display:none}}@media(max-width:979.98px){.navbar-expand-lg>.container,.navbar-expand-lg>.container-fluid,.navbar-expand-lg>.container-lg,.navbar-expand-lg>.container-md,.navbar-expand-lg>.container-sm,.navbar-expand-lg>.container-xl,.navbar-expand-lg>.container-xs{padding-right:0;padding-left:0}}@media(min-width:980px){.navbar-expand-lg{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-lg .navbar-nav{flex-direction:row}.navbar-expand-lg .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-lg .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-lg>.container,.navbar-expand-lg>.container-fluid,.navbar-expand-lg>.container-lg,.navbar-expand-lg>.container-md,.navbar-expand-lg>.container-sm,.navbar-expand-lg>.container-xl,.navbar-expand-lg>.container-xs{flex-wrap:nowrap}.navbar-expand-lg .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-lg .navbar-toggler{display:none}}@media(max-width:1239.98px){.navbar-expand-xl>.container,.navbar-expand-xl>.container-fluid,.navbar-expand-xl>.container-lg,.navbar-expand-xl>.container-md,.navbar-expand-xl>.container-sm,.navbar-expand-xl>.container-xl,.navbar-expand-xl>.container-xs{padding-right:0;padding-left:0}}@media(min-width:1240px){.navbar-expand-xl{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-xl .navbar-nav{flex-direction:row}.navbar-expand-xl .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-xl .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-xl>.container,.navbar-expand-xl>.container-fluid,.navbar-expand-xl>.container-lg,.navbar-expand-xl>.container-md,.navbar-expand-xl>.container-sm,.navbar-expand-xl>.container-xl,.navbar-expand-xl>.container-xs{flex-wrap:nowrap}.navbar-expand-xl .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-xl .navbar-toggler{display:none}}.navbar-expand{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand>.container,.navbar-expand>.container-fluid,.navbar-expand>.container-lg,.navbar-expand>.container-md,.navbar-expand>.container-sm,.navbar-expand>.container-xl,.navbar-expand>.container-xs{padding-right:0;padding-left:0}.navbar-expand .navbar-nav{flex-direction:row}.navbar-expand .navbar-nav .dropdown-menu{position:absolute}.navbar-expand .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand>.container,.navbar-expand>.container-fluid,.navbar-expand>.container-lg,.navbar-expand>.container-md,.navbar-expand>.container-sm,.navbar-expand>.container-xl,.navbar-expand>.container-xs{flex-wrap:nowrap}.navbar-expand .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand .navbar-toggler{display:none}.navbar-light .navbar-brand,.navbar-light .navbar-brand:focus,.navbar-light .navbar-brand:hover{color:rgba(33,37,41,.9)}.navbar-light .navbar-nav .nav-link{color:rgba(33,37,41,.5)}.navbar-light .navbar-nav .nav-link:focus,.navbar-light .navbar-nav .nav-link:hover{color:rgba(33,37,41,.7)}.navbar-light .navbar-nav .nav-link.disabled{color:rgba(33,37,41,.3)}.navbar-light .navbar-nav .active>.nav-link,.navbar-light .navbar-nav .nav-link.active,.navbar-light .navbar-nav .nav-link.show,.navbar-light .navbar-nav .show>.nav-link{color:rgba(33,37,41,.9)}.navbar-light .navbar-toggler{color:rgba(33,37,41,.5);border-color:rgba(33,37,41,.1)}.navbar-light .navbar-toggler-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(33, 37, 41, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")}.navbar-light .navbar-text{color:rgba(33,37,41,.5)}.navbar-light .navbar-text a,.navbar-light .navbar-text a:focus,.navbar-light .navbar-text a:hover{color:rgba(33,37,41,.9)}.navbar-dark .navbar-brand,.navbar-dark .navbar-brand:focus,.navbar-dark .navbar-brand:hover{color:#fff}.navbar-dark .navbar-nav .nav-link{color:hsla(0,0%,100%,.5)}.navbar-dark .navbar-nav .nav-link:focus,.navbar-dark .navbar-nav .nav-link:hover{color:hsla(0,0%,100%,.75)}.navbar-dark .navbar-nav .nav-link.disabled{color:hsla(0,0%,100%,.25)}.navbar-dark .navbar-nav .active>.nav-link,.navbar-dark .navbar-nav .nav-link.active,.navbar-dark .navbar-nav .nav-link.show,.navbar-dark .navbar-nav .show>.nav-link{color:#fff}.navbar-dark .navbar-toggler{color:hsla(0,0%,100%,.5);border-color:hsla(0,0%,100%,.1)}.navbar-dark .navbar-toggler-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(255, 255, 255, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")}.navbar-dark .navbar-text{color:hsla(0,0%,100%,.5)}.navbar-dark .navbar-text a,.navbar-dark .navbar-text a:focus,.navbar-dark .navbar-text a:hover{color:#fff}.card{position:relative;display:flex;flex-direction:column;min-width:0;word-wrap:break-word;background-color:#fff;background-clip:border-box;border:1px solid #d6dbdf;border-radius:8px}.card>hr{margin-right:0;margin-left:0}.card>.list-group:first-child .list-group-item:first-child{border-top-left-radius:8px;border-top-right-radius:8px}.card>.list-group:last-child .list-group-item:last-child{border-bottom-right-radius:8px;border-bottom-left-radius:8px}.card-body{flex:1 1 auto;min-height:1px;padding:24px}.card-title{margin-bottom:24px}.card-subtitle{margin-top:-12px}.card-subtitle,.card-text:last-child{margin-bottom:0}.card-link:hover{text-decoration:none}.card-link+.card-link{margin-left:24px}.card-header{padding:24px;margin-bottom:0;background-color:#f1f6f9;border-bottom:1px solid #d6dbdf}.card-header:first-child{border-radius:subtract(8px,1px) subtract(8px,1px) 0 0}.card-header+.list-group .list-group-item:first-child{border-top:0}.card-footer{padding:24px;background-color:#f1f6f9;border-top:1px solid #d6dbdf}.card-footer:last-child{border-radius:0 0 subtract(8px,1px) subtract(8px,1px)}.card-header-tabs{margin-bottom:-24px;border-bottom:0}.card-header-pills,.card-header-tabs{margin-right:-12px;margin-left:-12px}.card-img-overlay{position:absolute;top:0;right:0;bottom:0;left:0;padding:24px}.card-img,.card-img-bottom,.card-img-top{flex-shrink:0;width:100%}.card-img,.card-img-top{border-top-left-radius:subtract(8px,1px);border-top-right-radius:subtract(8px,1px)}.card-img,.card-img-bottom{border-bottom-right-radius:subtract(8px,1px);border-bottom-left-radius:subtract(8px,1px)}.card-deck .card{margin-bottom:20px}@media(min-width:616px){.card-deck{display:flex;flex-flow:row wrap;margin-right:-20px;margin-left:-20px}.card-deck .card{flex:1 0 0%;margin-right:20px;margin-bottom:0;margin-left:20px}}.card-group>.card{margin-bottom:20px}@media(min-width:616px){.card-group{display:flex;flex-flow:row wrap}.card-group>.card{flex:1 0 0%;margin-bottom:0}.card-group>.card+.card{margin-left:0;border-left:0}.card-group>.card:not(:last-child){border-top-right-radius:0;border-bottom-right-radius:0}.card-group>.card:not(:last-child) .card-header,.card-group>.card:not(:last-child) .card-img-top{border-top-right-radius:0}.card-group>.card:not(:last-child) .card-footer,.card-group>.card:not(:last-child) .card-img-bottom{border-bottom-right-radius:0}.card-group>.card:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.card-group>.card:not(:first-child) .card-header,.card-group>.card:not(:first-child) .card-img-top{border-top-left-radius:0}.card-group>.card:not(:first-child) .card-footer,.card-group>.card:not(:first-child) .card-img-bottom{border-bottom-left-radius:0}}.card-columns .card{margin-bottom:40px}@media(min-width:616px){.card-columns{-moz-column-count:3;column-count:3;-moz-column-gap:40px;column-gap:40px;orphans:1;widows:1}.card-columns .card{display:inline-block;width:100%}}.accordion>.card{overflow:hidden}.accordion>.card:not(:last-of-type){border-bottom:0;border-bottom-right-radius:0;border-bottom-left-radius:0}.accordion>.card:not(:first-of-type){border-top-left-radius:0;border-top-right-radius:0}.accordion>.card>.card-header{border-radius:0;margin-bottom:-1px}.breadcrumb{display:flex;flex-wrap:wrap;padding:.75rem 1rem;margin-bottom:1rem;list-style:none;background-color:#e9ecef;border-radius:8px}.breadcrumb-item+.breadcrumb-item{padding-left:.5rem}.breadcrumb-item+.breadcrumb-item:before{display:inline-block;padding-right:.5rem;color:#6c757d;content:"/"}.breadcrumb-item+.breadcrumb-item:hover:before{text-decoration:underline;text-decoration:none}.breadcrumb-item.active{color:#6c757d}.pagination{display:flex;padding-left:0;list-style:none;border-radius:8px}.page-link{position:relative;display:block;padding:.5rem .75rem;margin-left:-1px;line-height:1.25;color:#ff8c00;background-color:#fff;border:1px solid #dee2e6}.page-link:hover{z-index:2;color:#ff8c00;text-decoration:none;background-color:#e9ecef;border-color:#dee2e6}.page-link:focus{z-index:3;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.page-item:first-child .page-link{margin-left:0;border-top-left-radius:8px;border-bottom-left-radius:8px}.page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.page-item.active .page-link{z-index:3;color:#fff;background-color:#fc0;border-color:#fc0}.page-item.disabled .page-link{color:#6c757d;pointer-events:none;cursor:auto;background-color:#fff;border-color:#dee2e6}.pagination-lg .page-link{padding:.75rem 1.5rem;font-size:1.125rem;line-height:1.5}.pagination-lg .page-item:first-child .page-link{border-top-left-radius:8px;border-bottom-left-radius:8px}.pagination-lg .page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.pagination-sm .page-link{padding:.25rem .5rem;font-size:.875rem;line-height:1.5}.pagination-sm .page-item:first-child .page-link{border-top-left-radius:8px;border-bottom-left-radius:8px}.pagination-sm .page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.badge{display:inline-block;padding:.25em .4em;font-size:75%;font-weight:700;line-height:1;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:8px;transition:color .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.badge{transition:none}}a.badge:focus,a.badge:hover{text-decoration:none}.badge:empty{display:none}.btn .badge{position:relative;top:-1px}.badge-pill{padding-right:.6em;padding-left:.6em;border-radius:10rem}.badge-primary{color:#495057;background-color:#fc0}a.badge-primary:focus,a.badge-primary:hover{color:#495057;background-color:#cca300}a.badge-primary.focus,a.badge-primary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.5)}.badge-secondary{color:#fff;background-color:#212529}a.badge-secondary:focus,a.badge-secondary:hover{color:#fff;background-color:#0a0c0d}a.badge-secondary.focus,a.badge-secondary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(33,37,41,.5)}.badge-success{color:#fff;background-color:#28a745}a.badge-success:focus,a.badge-success:hover{color:#fff;background-color:#1e7e34}a.badge-success.focus,a.badge-success:focus{outline:0;box-shadow:0 0 0 .2rem rgba(40,167,69,.5)}.badge-info{color:#fff;background-color:#17a2b8}a.badge-info:focus,a.badge-info:hover{color:#fff;background-color:#117a8b}a.badge-info.focus,a.badge-info:focus{outline:0;box-shadow:0 0 0 .2rem rgba(23,162,184,.5)}.badge-warning{color:#495057;background-color:#ffc107}a.badge-warning:focus,a.badge-warning:hover{color:#495057;background-color:#d39e00}a.badge-warning.focus,a.badge-warning:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,193,7,.5)}.badge-danger{color:#fff;background-color:#dc3545}a.badge-danger:focus,a.badge-danger:hover{color:#fff;background-color:#bd2130}a.badge-danger.focus,a.badge-danger:focus{outline:0;box-shadow:0 0 0 .2rem rgba(220,53,69,.5)}.badge-light{color:#495057;background-color:#f1f6f9}a.badge-light:focus,a.badge-light:hover{color:#495057;background-color:#cddfea}a.badge-light.focus,a.badge-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(241,246,249,.5)}.badge-dark{color:#fff;background-color:#495057}a.badge-dark:focus,a.badge-dark:hover{color:#fff;background-color:#32373b}a.badge-dark.focus,a.badge-dark:focus{outline:0;box-shadow:0 0 0 .2rem rgba(73,80,87,.5)}.badge-primary-light{color:#495057;background-color:#fffaf0}a.badge-primary-light:focus,a.badge-primary-light:hover{color:#495057;background-color:#ffe9bd}a.badge-primary-light.focus,a.badge-primary-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,250,240,.5)}.badge-secondary-light{color:#495057;background-color:#fff}a.badge-secondary-light:focus,a.badge-secondary-light:hover{color:#495057;background-color:#e6e6e6}a.badge-secondary-light.focus,a.badge-secondary-light:focus{outline:0;box-shadow:0 0 0 .2rem hsla(0,0%,100%,.5)}.badge-tertiary{color:#fff;background-color:#257af4}a.badge-tertiary:focus,a.badge-tertiary:hover{color:#fff;background-color:#0b60db}a.badge-tertiary.focus,a.badge-tertiary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(37,122,244,.5)}.badge-tertiary-light{color:#495057;background-color:#e3f1fe}a.badge-tertiary-light:focus,a.badge-tertiary-light:hover{color:#495057;background-color:#b2d8fc}a.badge-tertiary-light.focus,a.badge-tertiary-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(227,241,254,.5)}.badge-white{color:#495057;background-color:#fff}a.badge-white:focus,a.badge-white:hover{color:#495057;background-color:#e6e6e6}a.badge-white.focus,a.badge-white:focus{outline:0;box-shadow:0 0 0 .2rem hsla(0,0%,100%,.5)}.badge-black{color:#fff;background-color:#212529}a.badge-black:focus,a.badge-black:hover{color:#fff;background-color:#0a0c0d}a.badge-black.focus,a.badge-black:focus{outline:0;box-shadow:0 0 0 .2rem rgba(33,37,41,.5)}.badge-blue{color:#fff;background-color:#257af4}a.badge-blue:focus,a.badge-blue:hover{color:#fff;background-color:#0b60db}a.badge-blue.focus,a.badge-blue:focus{outline:0;box-shadow:0 0 0 .2rem rgba(37,122,244,.5)}.badge-light-blue{color:#495057;background-color:#e3f1fe}a.badge-light-blue:focus,a.badge-light-blue:hover{color:#495057;background-color:#b2d8fc}a.badge-light-blue.focus,a.badge-light-blue:focus{outline:0;box-shadow:0 0 0 .2rem rgba(227,241,254,.5)}.badge-yellow{color:#495057;background-color:#fc0}a.badge-yellow:focus,a.badge-yellow:hover{color:#495057;background-color:#cca300}a.badge-yellow.focus,a.badge-yellow:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.5)}.badge-light-yellow{color:#495057;background-color:#fffaf0}a.badge-light-yellow:focus,a.badge-light-yellow:hover{color:#495057;background-color:#ffe9bd}a.badge-light-yellow.focus,a.badge-light-yellow:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,250,240,.5)}.badge-orange{color:#495057;background-color:#ff8c00}a.badge-orange:focus,a.badge-orange:hover{color:#495057;background-color:#cc7000}a.badge-orange.focus,a.badge-orange:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,140,0,.5)}.badge-light-orange{color:#495057;background-color:#ffe4b5}a.badge-light-orange:focus,a.badge-light-orange:hover{color:#495057;background-color:#ffd182}a.badge-light-orange.focus,a.badge-light-orange:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,228,181,.5)}.badge-red{color:#fff;background-color:#ff3939}a.badge-red:focus,a.badge-red:hover{color:#fff;background-color:#ff0606}a.badge-red.focus,a.badge-red:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,57,57,.5)}.badge-light-red{color:#495057;background-color:#ffe4e1}a.badge-light-red:focus,a.badge-light-red:hover{color:#495057;background-color:#ffb6ae}a.badge-light-red.focus,a.badge-light-red:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,228,225,.5)}.badge-medium{color:#495057;background-color:#d6dbdf}a.badge-medium:focus,a.badge-medium:hover{color:#495057;background-color:#b9c2c9}a.badge-medium.focus,a.badge-medium:focus{outline:0;box-shadow:0 0 0 .2rem rgba(214,219,223,.5)}.jumbotron{padding:2rem 1rem;margin-bottom:2rem;background-color:#e9ecef;border-radius:8px}@media(min-width:616px){.jumbotron{padding:4rem 2rem}}.jumbotron-fluid{padding-right:0;padding-left:0;border-radius:0}.alert{position:relative;padding:.75rem 1.25rem;margin-bottom:1rem;border:1px solid transparent;border-radius:8px}.alert-heading{color:inherit}.alert-link{font-weight:700}.alert-dismissible{padding-right:4rem}.alert-dismissible .close{position:absolute;top:0;right:0;padding:.75rem 1.25rem;color:inherit}.alert-primary{color:#947c14;background-color:#fff5cc;border-color:#fff1b8}.alert-primary hr{border-top-color:#ffec9f}.alert-primary .alert-link{color:#67560e}.alert-secondary{color:#212529;background-color:#d3d3d4;border-color:#c1c2c3}.alert-secondary hr{border-top-color:#b4b5b6}.alert-secondary .alert-link{color:#0a0c0d}.alert-success{color:#256938;background-color:#d4edda;border-color:#c3e6cb}.alert-success hr{border-top-color:#b1dfbb}.alert-success .alert-link{color:#184324}.alert-info{color:#1c6673;background-color:#d1ecf1;border-color:#bee5eb}.alert-info hr{border-top-color:#abdde5}.alert-info .alert-link{color:#12424a}.alert-warning{color:#947617;background-color:#fff3cd;border-color:#ffeeba}.alert-warning hr{border-top-color:#ffe8a1}.alert-warning .alert-link{color:#685310}.alert-danger{color:#822d38;background-color:#f8d7da;border-color:#f5c6cb}.alert-danger hr{border-top-color:#f1b0b7}.alert-danger .alert-link{color:#5c2028}.alert-light{color:#8d9295;background-color:#fcfdfe;border-color:#fbfcfd}.alert-light hr{border-top-color:#eaeff5}.alert-light .alert-link{color:#73797c}.alert-dark{color:#363b41;background-color:#dbdcdd;border-color:#ccced0}.alert-dark hr{border-top-color:#bfc1c4}.alert-dark .alert-link{color:#1f2225}.alert-primary-light{color:#949490;background-color:#fffefc;border-color:#fffefb}.alert-primary-light hr{border-top-color:#fff8e2}.alert-primary-light .alert-link{color:#7b7b76}.alert-secondary-light{color:#949698;background-color:#fff;border-color:#fff}.alert-secondary-light hr{border-top-color:#f2f2f2}.alert-secondary-light .alert-link{color:#7a7d7f}.alert-tertiary{color:#235193;background-color:#d3e4fd;border-color:#c2dafc}.alert-tertiary hr{border-top-color:#aacbfb}.alert-tertiary .alert-link{color:#193a6a}.alert-tertiary-light{color:#868f98;background-color:#f9fcff;border-color:#f7fbff}.alert-tertiary-light hr{border-top-color:#deeeff}.alert-tertiary-light .alert-link{color:#6c767f}.alert-white{color:#949698;background-color:#fff;border-color:#fff}.alert-white hr{border-top-color:#f2f2f2}.alert-white .alert-link{color:#7a7d7f}.alert-black{color:#212529;background-color:#d3d3d4;border-color:#c1c2c3}.alert-black hr{border-top-color:#b4b5b6}.alert-black .alert-link{color:#0a0c0d}.alert-blue{color:#235193;background-color:#d3e4fd;border-color:#c2dafc}.alert-blue hr{border-top-color:#aacbfb}.alert-blue .alert-link{color:#193a6a}.alert-light-blue{color:#868f98;background-color:#f9fcff;border-color:#f7fbff}.alert-light-blue hr{border-top-color:#deeeff}.alert-light-blue .alert-link{color:#6c767f}.alert-yellow{color:#947c14;background-color:#fff5cc;border-color:#fff1b8}.alert-yellow hr{border-top-color:#ffec9f}.alert-yellow .alert-link{color:#67560e}.alert-light-yellow{color:#949490;background-color:#fffefc;border-color:#fffefb}.alert-light-yellow hr{border-top-color:#fff8e2}.alert-light-yellow .alert-link{color:#7b7b76}.alert-orange{color:#945b14;background-color:#ffe8cc;border-color:#ffdfb8}.alert-orange hr{border-top-color:#ffd49f}.alert-orange .alert-link{color:#673f0e}.alert-light-orange{color:#948872;background-color:#fffaf0;border-color:#fff7ea}.alert-light-orange hr{border-top-color:#ffedd1}.alert-light-orange .alert-link{color:#786e5b}.alert-red{color:#942f31;background-color:#ffd7d7;border-color:#ffc8c8}.alert-red hr{border-top-color:#ffafaf}.alert-red .alert-link{color:#6d2324}.alert-light-red{color:#948889;background-color:#fffaf9;border-color:#fff7f7}.alert-light-red hr{border-top-color:#ffdede}.alert-light-red .alert-link{color:#7b6e6f}.alert-medium{color:#7f8488;background-color:#f7f8f9;border-color:#f4f5f6}.alert-medium hr{border-top-color:#e6e8eb}.alert-medium .alert-link{color:#666a6e}@-webkit-keyframes progress-bar-stripes{0%{background-position:1rem 0}to{background-position:0 0}}@keyframes progress-bar-stripes{0%{background-position:1rem 0}to{background-position:0 0}}.progress{height:1rem;font-size:.75rem;background-color:#e9ecef;border-radius:8px}.progress,.progress-bar{display:flex;overflow:hidden}.progress-bar{flex-direction:column;justify-content:center;color:#fff;text-align:center;white-space:nowrap;background-color:#fc0;transition:width .6s ease}@media(prefers-reduced-motion:reduce){.progress-bar{transition:none}}.progress-bar-striped{background-image:linear-gradient(45deg,hsla(0,0%,100%,.15) 25%,transparent 0,transparent 50%,hsla(0,0%,100%,.15) 0,hsla(0,0%,100%,.15) 75%,transparent 0,transparent);background-size:1rem 1rem}.progress-bar-animated{-webkit-animation:progress-bar-stripes 1s linear infinite;animation:progress-bar-stripes 1s linear infinite}@media(prefers-reduced-motion:reduce){.progress-bar-animated{-webkit-animation:none;animation:none}}.media{display:flex;align-items:flex-start}.media-body{flex:1}.list-group{display:flex;flex-direction:column;padding-left:0;margin-bottom:0}.list-group-item-action{width:100%;color:#6c757d;text-align:inherit}.list-group-item-action:focus,.list-group-item-action:hover{z-index:1;color:#6c757d;text-decoration:none;background-color:#f1f6f9}.list-group-item-action:active{color:#212529;background-color:#e9ecef}.list-group-item{position:relative;display:block;padding:.75rem 1.25rem;background-color:#fff;border:1px solid rgba(33,37,41,.125)}.list-group-item:first-child{border-top-left-radius:8px;border-top-right-radius:8px}.list-group-item:last-child{border-bottom-right-radius:8px;border-bottom-left-radius:8px}.list-group-item.disabled,.list-group-item:disabled{color:#6c757d;pointer-events:none;background-color:#fff}.list-group-item.active{z-index:2;color:#fff;background-color:#fc0;border-color:#fc0}.list-group-item+.list-group-item{border-top-width:0}.list-group-item+.list-group-item.active{margin-top:-1px;border-top-width:1px}.list-group-horizontal{flex-direction:row}.list-group-horizontal .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal .list-group-item.active{margin-top:0}.list-group-horizontal .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}@media(min-width:400px){.list-group-horizontal-xs{flex-direction:row}.list-group-horizontal-xs .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-xs .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-xs .list-group-item.active{margin-top:0}.list-group-horizontal-xs .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-xs .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:616px){.list-group-horizontal-sm{flex-direction:row}.list-group-horizontal-sm .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-sm .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-sm .list-group-item.active{margin-top:0}.list-group-horizontal-sm .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-sm .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:768px){.list-group-horizontal-md{flex-direction:row}.list-group-horizontal-md .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-md .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-md .list-group-item.active{margin-top:0}.list-group-horizontal-md .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-md .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:980px){.list-group-horizontal-lg{flex-direction:row}.list-group-horizontal-lg .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-lg .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-lg .list-group-item.active{margin-top:0}.list-group-horizontal-lg .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-lg .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:1240px){.list-group-horizontal-xl{flex-direction:row}.list-group-horizontal-xl .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-xl .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-xl .list-group-item.active{margin-top:0}.list-group-horizontal-xl .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-xl .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}.list-group-flush .list-group-item{border-right-width:0;border-left-width:0;border-radius:0}.list-group-flush .list-group-item:first-child{border-top-width:0}.list-group-flush:last-child .list-group-item:last-child{border-bottom-width:0}.list-group-item-primary{color:#947c14;background-color:#fff1b8}.list-group-item-primary.list-group-item-action:focus,.list-group-item-primary.list-group-item-action:hover{color:#947c14;background-color:#ffec9f}.list-group-item-primary.list-group-item-action.active{color:#fff;background-color:#947c14;border-color:#947c14}.list-group-item-secondary{color:#212529;background-color:#c1c2c3}.list-group-item-secondary.list-group-item-action:focus,.list-group-item-secondary.list-group-item-action:hover{color:#212529;background-color:#b4b5b6}.list-group-item-secondary.list-group-item-action.active{color:#fff;background-color:#212529;border-color:#212529}.list-group-item-success{color:#256938;background-color:#c3e6cb}.list-group-item-success.list-group-item-action:focus,.list-group-item-success.list-group-item-action:hover{color:#256938;background-color:#b1dfbb}.list-group-item-success.list-group-item-action.active{color:#fff;background-color:#256938;border-color:#256938}.list-group-item-info{color:#1c6673;background-color:#bee5eb}.list-group-item-info.list-group-item-action:focus,.list-group-item-info.list-group-item-action:hover{color:#1c6673;background-color:#abdde5}.list-group-item-info.list-group-item-action.active{color:#fff;background-color:#1c6673;border-color:#1c6673}.list-group-item-warning{color:#947617;background-color:#ffeeba}.list-group-item-warning.list-group-item-action:focus,.list-group-item-warning.list-group-item-action:hover{color:#947617;background-color:#ffe8a1}.list-group-item-warning.list-group-item-action.active{color:#fff;background-color:#947617;border-color:#947617}.list-group-item-danger{color:#822d38;background-color:#f5c6cb}.list-group-item-danger.list-group-item-action:focus,.list-group-item-danger.list-group-item-action:hover{color:#822d38;background-color:#f1b0b7}.list-group-item-danger.list-group-item-action.active{color:#fff;background-color:#822d38;border-color:#822d38}.list-group-item-light{color:#8d9295;background-color:#fbfcfd}.list-group-item-light.list-group-item-action:focus,.list-group-item-light.list-group-item-action:hover{color:#8d9295;background-color:#eaeff5}.list-group-item-light.list-group-item-action.active{color:#fff;background-color:#8d9295;border-color:#8d9295}.list-group-item-dark{color:#363b41;background-color:#ccced0}.list-group-item-dark.list-group-item-action:focus,.list-group-item-dark.list-group-item-action:hover{color:#363b41;background-color:#bfc1c4}.list-group-item-dark.list-group-item-action.active{color:#fff;background-color:#363b41;border-color:#363b41}.list-group-item-primary-light{color:#949490;background-color:#fffefb}.list-group-item-primary-light.list-group-item-action:focus,.list-group-item-primary-light.list-group-item-action:hover{color:#949490;background-color:#fff8e2}.list-group-item-primary-light.list-group-item-action.active{color:#fff;background-color:#949490;border-color:#949490}.list-group-item-secondary-light{color:#949698;background-color:#fff}.list-group-item-secondary-light.list-group-item-action:focus,.list-group-item-secondary-light.list-group-item-action:hover{color:#949698;background-color:#f2f2f2}.list-group-item-secondary-light.list-group-item-action.active{color:#fff;background-color:#949698;border-color:#949698}.list-group-item-tertiary{color:#235193;background-color:#c2dafc}.list-group-item-tertiary.list-group-item-action:focus,.list-group-item-tertiary.list-group-item-action:hover{color:#235193;background-color:#aacbfb}.list-group-item-tertiary.list-group-item-action.active{color:#fff;background-color:#235193;border-color:#235193}.list-group-item-tertiary-light{color:#868f98;background-color:#f7fbff}.list-group-item-tertiary-light.list-group-item-action:focus,.list-group-item-tertiary-light.list-group-item-action:hover{color:#868f98;background-color:#deeeff}.list-group-item-tertiary-light.list-group-item-action.active{color:#fff;background-color:#868f98;border-color:#868f98}.list-group-item-white{color:#949698;background-color:#fff}.list-group-item-white.list-group-item-action:focus,.list-group-item-white.list-group-item-action:hover{color:#949698;background-color:#f2f2f2}.list-group-item-white.list-group-item-action.active{color:#fff;background-color:#949698;border-color:#949698}.list-group-item-black{color:#212529;background-color:#c1c2c3}.list-group-item-black.list-group-item-action:focus,.list-group-item-black.list-group-item-action:hover{color:#212529;background-color:#b4b5b6}.list-group-item-black.list-group-item-action.active{color:#fff;background-color:#212529;border-color:#212529}.list-group-item-blue{color:#235193;background-color:#c2dafc}.list-group-item-blue.list-group-item-action:focus,.list-group-item-blue.list-group-item-action:hover{color:#235193;background-color:#aacbfb}.list-group-item-blue.list-group-item-action.active{color:#fff;background-color:#235193;border-color:#235193}.list-group-item-light-blue{color:#868f98;background-color:#f7fbff}.list-group-item-light-blue.list-group-item-action:focus,.list-group-item-light-blue.list-group-item-action:hover{color:#868f98;background-color:#deeeff}.list-group-item-light-blue.list-group-item-action.active{color:#fff;background-color:#868f98;border-color:#868f98}.list-group-item-yellow{color:#947c14;background-color:#fff1b8}.list-group-item-yellow.list-group-item-action:focus,.list-group-item-yellow.list-group-item-action:hover{color:#947c14;background-color:#ffec9f}.list-group-item-yellow.list-group-item-action.active{color:#fff;background-color:#947c14;border-color:#947c14}.list-group-item-light-yellow{color:#949490;background-color:#fffefb}.list-group-item-light-yellow.list-group-item-action:focus,.list-group-item-light-yellow.list-group-item-action:hover{color:#949490;background-color:#fff8e2}.list-group-item-light-yellow.list-group-item-action.active{color:#fff;background-color:#949490;border-color:#949490}.list-group-item-orange{color:#945b14;background-color:#ffdfb8}.list-group-item-orange.list-group-item-action:focus,.list-group-item-orange.list-group-item-action:hover{color:#945b14;background-color:#ffd49f}.list-group-item-orange.list-group-item-action.active{color:#fff;background-color:#945b14;border-color:#945b14}.list-group-item-light-orange{color:#948872;background-color:#fff7ea}.list-group-item-light-orange.list-group-item-action:focus,.list-group-item-light-orange.list-group-item-action:hover{color:#948872;background-color:#ffedd1}.list-group-item-light-orange.list-group-item-action.active{color:#fff;background-color:#948872;border-color:#948872}.list-group-item-red{color:#942f31;background-color:#ffc8c8}.list-group-item-red.list-group-item-action:focus,.list-group-item-red.list-group-item-action:hover{color:#942f31;background-color:#ffafaf}.list-group-item-red.list-group-item-action.active{color:#fff;background-color:#942f31;border-color:#942f31}.list-group-item-light-red{color:#948889;background-color:#fff7f7}.list-group-item-light-red.list-group-item-action:focus,.list-group-item-light-red.list-group-item-action:hover{color:#948889;background-color:#ffdede}.list-group-item-light-red.list-group-item-action.active{color:#fff;background-color:#948889;border-color:#948889}.list-group-item-medium{color:#7f8488;background-color:#f4f5f6}.list-group-item-medium.list-group-item-action:focus,.list-group-item-medium.list-group-item-action:hover{color:#7f8488;background-color:#e6e8eb}.list-group-item-medium.list-group-item-action.active{color:#fff;background-color:#7f8488;border-color:#7f8488}.close{float:right;font-size:1.5rem;font-weight:700;line-height:1;color:#212529;text-shadow:0 1px 0 #fff;opacity:.5}@media(max-width:1200px){.close{font-size:calc(1.275rem + .3vw)}}.close:hover{color:#212529;text-decoration:none}.close:not(:disabled):not(.disabled):focus,.close:not(:disabled):not(.disabled):hover{opacity:.75}button.close{padding:0;background-color:transparent;border:0;-webkit-appearance:none;-moz-appearance:none;appearance:none}a.close.disabled{pointer-events:none}.toast{max-width:350px;overflow:hidden;font-size:.875rem;background-color:hsla(0,0%,100%,.85);background-clip:padding-box;border:1px solid rgba(0,0,0,.1);box-shadow:0 .25rem .75rem rgba(33,37,41,.1);-webkit-backdrop-filter:blur(10px);backdrop-filter:blur(10px);opacity:0;border-radius:.25rem}.toast:not(:last-child){margin-bottom:.75rem}.toast.showing{opacity:1}.toast.show{display:block;opacity:1}.toast.hide{display:none}.toast-header{display:flex;align-items:center;padding:.25rem .75rem;color:#6c757d;background-color:hsla(0,0%,100%,.85);background-clip:padding-box;border-bottom:1px solid rgba(0,0,0,.05)}.toast-body{padding:.75rem}.modal-open{overflow:hidden}.modal-open .modal{overflow-x:hidden;overflow-y:auto}.modal{position:fixed;top:0;left:0;z-index:1050;display:none;width:100%;height:100%;overflow:hidden;outline:0}.modal-dialog{position:relative;width:auto;margin:.5rem;pointer-events:none}.modal.fade .modal-dialog{transition:transform .3s ease-out;transform:translateY(-50px)}@media(prefers-reduced-motion:reduce){.modal.fade .modal-dialog{transition:none}}.modal.show .modal-dialog{transform:none}.modal.modal-static .modal-dialog{transform:scale(1.02)}.modal-dialog-scrollable{display:flex;max-height:calc(100% - 1rem)}.modal-dialog-scrollable .modal-content{max-height:calc(100vh - 1rem);overflow:hidden}.modal-dialog-scrollable .modal-footer,.modal-dialog-scrollable .modal-header{flex-shrink:0}.modal-dialog-scrollable .modal-body{overflow-y:auto}.modal-dialog-centered{display:flex;align-items:center;min-height:calc(100% - 1rem)}.modal-dialog-centered:before{display:block;height:calc(100vh - 1rem);content:""}.modal-dialog-centered.modal-dialog-scrollable{flex-direction:column;justify-content:center;height:100%}.modal-dialog-centered.modal-dialog-scrollable .modal-content{max-height:none}.modal-dialog-centered.modal-dialog-scrollable:before{content:none}.modal-content{position:relative;display:flex;flex-direction:column;width:100%;pointer-events:auto;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.2);border-radius:8px;outline:0}.modal-backdrop{position:fixed;top:0;left:0;z-index:1040;width:100vw;height:100vh;background-color:#212529}.modal-backdrop.fade{opacity:0}.modal-backdrop.show{opacity:.5}.modal-header{display:flex;align-items:flex-start;justify-content:space-between;padding:1rem;border-bottom:1px solid #d6dbdf;border-top-left-radius:7px;border-top-right-radius:7px}.modal-header .close{padding:1rem;margin:-1rem -1rem -1rem auto}.modal-title{margin-bottom:0;line-height:1.5}.modal-body{position:relative;flex:1 1 auto;padding:1rem}.modal-footer{display:flex;flex-wrap:wrap;align-items:center;justify-content:flex-end;padding:.75rem;border-top:1px solid #d6dbdf;border-bottom-right-radius:7px;border-bottom-left-radius:7px}.modal-footer>*{margin:.25rem}.modal-scrollbar-measure{position:absolute;top:-9999px;width:50px;height:50px;overflow:scroll}@media(min-width:616px){.modal-dialog{max-width:500px;margin:1.75rem auto}.modal-dialog-scrollable{max-height:calc(100% - 3.5rem)}.modal-dialog-scrollable .modal-content{max-height:calc(100vh - 3.5rem)}.modal-dialog-centered{min-height:calc(100% - 3.5rem)}.modal-dialog-centered:before{height:calc(100vh - 3.5rem)}.modal-sm{max-width:300px}}@media(min-width:980px){.modal-lg,.modal-xl{max-width:800px}}@media(min-width:1240px){.modal-xl{max-width:1140px}}.tooltip{position:absolute;z-index:1070;display:block;margin:0;font-family:Noto Sans,sans-serif;font-style:normal;font-weight:400;line-height:1.5;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;letter-spacing:normal;word-break:normal;word-spacing:normal;white-space:normal;line-break:auto;font-size:.875rem;word-wrap:break-word;opacity:0}.tooltip.show{opacity:.9}.tooltip .arrow{position:absolute;display:block;width:.8rem;height:.4rem}.tooltip .arrow:before{position:absolute;content:"";border-color:transparent;border-style:solid}.bs-tooltip-auto[x-placement^=top],.bs-tooltip-top{padding:.4rem 0}.bs-tooltip-auto[x-placement^=top] .arrow,.bs-tooltip-top .arrow{bottom:0}.bs-tooltip-auto[x-placement^=top] .arrow:before,.bs-tooltip-top .arrow:before{top:0;border-width:.4rem .4rem 0;border-top-color:#212529}.bs-tooltip-auto[x-placement^=right],.bs-tooltip-right{padding:0 .4rem}.bs-tooltip-auto[x-placement^=right] .arrow,.bs-tooltip-right .arrow{left:0;width:.4rem;height:.8rem}.bs-tooltip-auto[x-placement^=right] .arrow:before,.bs-tooltip-right .arrow:before{right:0;border-width:.4rem .4rem .4rem 0;border-right-color:#212529}.bs-tooltip-auto[x-placement^=bottom],.bs-tooltip-bottom{padding:.4rem 0}.bs-tooltip-auto[x-placement^=bottom] .arrow,.bs-tooltip-bottom .arrow{top:0}.bs-tooltip-auto[x-placement^=bottom] .arrow:before,.bs-tooltip-bottom .arrow:before{bottom:0;border-width:0 .4rem .4rem;border-bottom-color:#212529}.bs-tooltip-auto[x-placement^=left],.bs-tooltip-left{padding:0 .4rem}.bs-tooltip-auto[x-placement^=left] .arrow,.bs-tooltip-left .arrow{right:0;width:.4rem;height:.8rem}.bs-tooltip-auto[x-placement^=left] .arrow:before,.bs-tooltip-left .arrow:before{left:0;border-width:.4rem 0 .4rem .4rem;border-left-color:#212529}.tooltip-inner{max-width:200px;padding:.25rem .5rem;color:#fff;text-align:center;background-color:#212529;border-radius:8px}.popover{top:0;left:0;z-index:1060;max-width:276px;font-family:Noto Sans,sans-serif;font-style:normal;font-weight:400;line-height:1.5;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;letter-spacing:normal;word-break:normal;word-spacing:normal;white-space:normal;line-break:auto;font-size:.875rem;word-wrap:break-word;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.2);border-radius:8px}.popover,.popover .arrow{position:absolute;display:block}.popover .arrow{width:1rem;height:.5rem;margin:0 8px}.popover .arrow:after,.popover .arrow:before{position:absolute;display:block;content:"";border-color:transparent;border-style:solid}.bs-popover-auto[x-placement^=top],.bs-popover-top{margin-bottom:.5rem}.bs-popover-auto[x-placement^=top]>.arrow,.bs-popover-top>.arrow{bottom:calc(-.5rem - 1px)}.bs-popover-auto[x-placement^=top]>.arrow:before,.bs-popover-top>.arrow:before{bottom:0;border-width:.5rem .5rem 0;border-top-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=top]>.arrow:after,.bs-popover-top>.arrow:after{bottom:1px;border-width:.5rem .5rem 0;border-top-color:#fff}.bs-popover-auto[x-placement^=right],.bs-popover-right{margin-left:.5rem}.bs-popover-auto[x-placement^=right]>.arrow,.bs-popover-right>.arrow{left:calc(-.5rem - 1px);width:.5rem;height:1rem;margin:8px 0}.bs-popover-auto[x-placement^=right]>.arrow:before,.bs-popover-right>.arrow:before{left:0;border-width:.5rem .5rem .5rem 0;border-right-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=right]>.arrow:after,.bs-popover-right>.arrow:after{left:1px;border-width:.5rem .5rem .5rem 0;border-right-color:#fff}.bs-popover-auto[x-placement^=bottom],.bs-popover-bottom{margin-top:.5rem}.bs-popover-auto[x-placement^=bottom]>.arrow,.bs-popover-bottom>.arrow{top:calc(-.5rem - 1px)}.bs-popover-auto[x-placement^=bottom]>.arrow:before,.bs-popover-bottom>.arrow:before{top:0;border-width:0 .5rem .5rem;border-bottom-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=bottom]>.arrow:after,.bs-popover-bottom>.arrow:after{top:1px;border-width:0 .5rem .5rem;border-bottom-color:#fff}.bs-popover-auto[x-placement^=bottom] .popover-header:before,.bs-popover-bottom .popover-header:before{position:absolute;top:0;left:50%;display:block;width:1rem;margin-left:-.5rem;content:"";border-bottom:1px solid #f7f7f7}.bs-popover-auto[x-placement^=left],.bs-popover-left{margin-right:.5rem}.bs-popover-auto[x-placement^=left]>.arrow,.bs-popover-left>.arrow{right:calc(-.5rem - 1px);width:.5rem;height:1rem;margin:8px 0}.bs-popover-auto[x-placement^=left]>.arrow:before,.bs-popover-left>.arrow:before{right:0;border-width:.5rem 0 .5rem .5rem;border-left-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=left]>.arrow:after,.bs-popover-left>.arrow:after{right:1px;border-width:.5rem 0 .5rem .5rem;border-left-color:#fff}.popover-header{padding:.5rem .75rem;margin-bottom:0;font-size:1rem;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;border-top-left-radius:7px;border-top-right-radius:7px}.popover-header:empty{display:none}.popover-body{padding:.5rem .75rem;color:#212529}.carousel{position:relative}.carousel.pointer-event{touch-action:pan-y}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner:after{display:block;clear:both;content:""}.carousel-item{position:relative;display:none;float:left;width:100%;margin-right:-100%;-webkit-backface-visibility:hidden;backface-visibility:hidden;transition:transform .6s ease-in-out}@media(prefers-reduced-motion:reduce){.carousel-item{transition:none}}.carousel-item-next,.carousel-item-prev,.carousel-item.active{display:block}.active.carousel-item-right,.carousel-item-next:not(.carousel-item-left){transform:translateX(100%)}.active.carousel-item-left,.carousel-item-prev:not(.carousel-item-right){transform:translateX(-100%)}.carousel-fade .carousel-item{opacity:0;transition-property:opacity;transform:none}.carousel-fade .carousel-item-next.carousel-item-left,.carousel-fade .carousel-item-prev.carousel-item-right,.carousel-fade .carousel-item.active{z-index:1;opacity:1}.carousel-fade .active.carousel-item-left,.carousel-fade .active.carousel-item-right{z-index:0;opacity:0;transition:opacity 0s .6s}@media(prefers-reduced-motion:reduce){.carousel-fade .active.carousel-item-left,.carousel-fade .active.carousel-item-right{transition:none}}.carousel-control-next,.carousel-control-prev{position:absolute;top:0;bottom:0;z-index:1;display:flex;align-items:center;justify-content:center;width:15%;color:#fff;text-align:center;opacity:.5;transition:opacity .15s ease}@media(prefers-reduced-motion:reduce){.carousel-control-next,.carousel-control-prev{transition:none}}.carousel-control-next:focus,.carousel-control-next:hover,.carousel-control-prev:focus,.carousel-control-prev:hover{color:#fff;text-decoration:none;outline:0;opacity:.9}.carousel-control-prev{left:0}.carousel-control-next{right:0}.carousel-control-next-icon,.carousel-control-prev-icon{display:inline-block;width:20px;height:20px;background:no-repeat 50%/100% 100%}.carousel-control-prev-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8'%3E%3Cpath d='M5.25 0l-4 4 4 4 1.5-1.5L4.25 4l2.5-2.5L5.25 0z'/%3E%3C/svg%3E")}.carousel-control-next-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8'%3E%3Cpath d='M2.75 0l-1.5 1.5L3.75 4l-2.5 2.5L2.75 8l4-4-4-4z'/%3E%3C/svg%3E")}.carousel-indicators{position:absolute;right:0;bottom:0;left:0;z-index:15;display:flex;justify-content:center;padding-left:0;margin-right:15%;margin-left:15%;list-style:none}.carousel-indicators li{box-sizing:content-box;flex:0 1 auto;width:30px;height:3px;margin-right:3px;margin-left:3px;text-indent:-999px;cursor:pointer;background-color:#fff;background-clip:padding-box;border-top:10px solid transparent;border-bottom:10px solid transparent;opacity:.5;transition:opacity .6s ease}@media(prefers-reduced-motion:reduce){.carousel-indicators li{transition:none}}.carousel-indicators .active{opacity:1}.carousel-caption{position:absolute;right:15%;bottom:20px;left:15%;z-index:10;padding-top:20px;padding-bottom:20px;color:#fff;text-align:center}@-webkit-keyframes spinner-border{to{transform:rotate(1turn)}}@keyframes spinner-border{to{transform:rotate(1turn)}}.spinner-border{display:inline-block;width:2rem;height:2rem;vertical-align:text-bottom;border:.25em solid;border-right:.25em solid transparent;border-radius:50%;-webkit-animation:spinner-border .75s linear infinite;animation:spinner-border .75s linear infinite}.spinner-border-sm{width:1rem;height:1rem;border-width:.2em}@-webkit-keyframes spinner-grow{0%{transform:scale(0)}50%{opacity:1}}@keyframes spinner-grow{0%{transform:scale(0)}50%{opacity:1}}.spinner-grow{display:inline-block;width:2rem;height:2rem;vertical-align:text-bottom;background-color:currentColor;border-radius:50%;opacity:0;-webkit-animation:spinner-grow .75s linear infinite;animation:spinner-grow .75s linear infinite}.spinner-grow-sm{width:1rem;height:1rem}.align-baseline{vertical-align:baseline!important}.align-top{vertical-align:top!important}.align-middle{vertical-align:middle!important}.align-bottom{vertical-align:bottom!important}.align-text-bottom{vertical-align:text-bottom!important}.align-text-top{vertical-align:text-top!important}.bg-primary{background-color:#fc0!important}a.bg-primary:focus,a.bg-primary:hover,button.bg-primary:focus,button.bg-primary:hover{background-color:#cca300!important}.bg-secondary{background-color:#212529!important}a.bg-secondary:focus,a.bg-secondary:hover,button.bg-secondary:focus,button.bg-secondary:hover{background-color:#0a0c0d!important}.bg-success{background-color:#28a745!important}a.bg-success:focus,a.bg-success:hover,button.bg-success:focus,button.bg-success:hover{background-color:#1e7e34!important}.bg-info{background-color:#17a2b8!important}a.bg-info:focus,a.bg-info:hover,button.bg-info:focus,button.bg-info:hover{background-color:#117a8b!important}.bg-warning{background-color:#ffc107!important}a.bg-warning:focus,a.bg-warning:hover,button.bg-warning:focus,button.bg-warning:hover{background-color:#d39e00!important}.bg-danger{background-color:#dc3545!important}a.bg-danger:focus,a.bg-danger:hover,button.bg-danger:focus,button.bg-danger:hover{background-color:#bd2130!important}.bg-light{background-color:#f1f6f9!important}a.bg-light:focus,a.bg-light:hover,button.bg-light:focus,button.bg-light:hover{background-color:#cddfea!important}.bg-dark{background-color:#495057!important}a.bg-dark:focus,a.bg-dark:hover,button.bg-dark:focus,button.bg-dark:hover{background-color:#32373b!important}.bg-primary-light{background-color:#fffaf0!important}a.bg-primary-light:focus,a.bg-primary-light:hover,button.bg-primary-light:focus,button.bg-primary-light:hover{background-color:#ffe9bd!important}.bg-secondary-light{background-color:#fff!important}a.bg-secondary-light:focus,a.bg-secondary-light:hover,button.bg-secondary-light:focus,button.bg-secondary-light:hover{background-color:#e6e6e6!important}.bg-tertiary{background-color:#257af4!important}a.bg-tertiary:focus,a.bg-tertiary:hover,button.bg-tertiary:focus,button.bg-tertiary:hover{background-color:#0b60db!important}.bg-tertiary-light{background-color:#e3f1fe!important}a.bg-tertiary-light:focus,a.bg-tertiary-light:hover,button.bg-tertiary-light:focus,button.bg-tertiary-light:hover{background-color:#b2d8fc!important}a.bg-white:focus,a.bg-white:hover,button.bg-white:focus,button.bg-white:hover{background-color:#e6e6e6!important}.bg-black{background-color:#212529!important}a.bg-black:focus,a.bg-black:hover,button.bg-black:focus,button.bg-black:hover{background-color:#0a0c0d!important}.bg-blue{background-color:#257af4!important}a.bg-blue:focus,a.bg-blue:hover,button.bg-blue:focus,button.bg-blue:hover{background-color:#0b60db!important}.bg-light-blue{background-color:#e3f1fe!important}a.bg-light-blue:focus,a.bg-light-blue:hover,button.bg-light-blue:focus,button.bg-light-blue:hover{background-color:#b2d8fc!important}.bg-yellow{background-color:#fc0!important}a.bg-yellow:focus,a.bg-yellow:hover,button.bg-yellow:focus,button.bg-yellow:hover{background-color:#cca300!important}.bg-light-yellow{background-color:#fffaf0!important}a.bg-light-yellow:focus,a.bg-light-yellow:hover,button.bg-light-yellow:focus,button.bg-light-yellow:hover{background-color:#ffe9bd!important}.bg-orange{background-color:#ff8c00!important}a.bg-orange:focus,a.bg-orange:hover,button.bg-orange:focus,button.bg-orange:hover{background-color:#cc7000!important}.bg-light-orange{background-color:#ffe4b5!important}a.bg-light-orange:focus,a.bg-light-orange:hover,button.bg-light-orange:focus,button.bg-light-orange:hover{background-color:#ffd182!important}.bg-red{background-color:#ff3939!important}a.bg-red:focus,a.bg-red:hover,button.bg-red:focus,button.bg-red:hover{background-color:#ff0606!important}.bg-light-red{background-color:#ffe4e1!important}a.bg-light-red:focus,a.bg-light-red:hover,button.bg-light-red:focus,button.bg-light-red:hover{background-color:#ffb6ae!important}.bg-medium{background-color:#d6dbdf!important}a.bg-medium:focus,a.bg-medium:hover,button.bg-medium:focus,button.bg-medium:hover{background-color:#b9c2c9!important}.bg-white{background-color:#fff!important}.bg-transparent{background-color:transparent!important}.border{border:1px solid #d6dbdf!important}.border-top{border-top:1px solid #d6dbdf!important}.border-right{border-right:1px solid #d6dbdf!important}.border-bottom{border-bottom:1px solid #d6dbdf!important}.border-left{border-left:1px solid #d6dbdf!important}.border-0{border:0!important}.border-top-0{border-top:0!important}.border-right-0{border-right:0!important}.border-bottom-0{border-bottom:0!important}.border-left-0{border-left:0!important}.border-primary{border-color:#fc0!important}.border-secondary{border-color:#212529!important}.border-success{border-color:#28a745!important}.border-info{border-color:#17a2b8!important}.border-warning{border-color:#ffc107!important}.border-danger{border-color:#dc3545!important}.border-light{border-color:#f1f6f9!important}.border-dark{border-color:#495057!important}.border-primary-light{border-color:#fffaf0!important}.border-secondary-light{border-color:#fff!important}.border-tertiary{border-color:#257af4!important}.border-tertiary-light{border-color:#e3f1fe!important}.border-black{border-color:#212529!important}.border-blue{border-color:#257af4!important}.border-light-blue{border-color:#e3f1fe!important}.border-yellow{border-color:#fc0!important}.border-light-yellow{border-color:#fffaf0!important}.border-orange{border-color:#ff8c00!important}.border-light-orange{border-color:#ffe4b5!important}.border-red{border-color:#ff3939!important}.border-light-red{border-color:#ffe4e1!important}.border-medium{border-color:#d6dbdf!important}.border-white{border-color:#fff!important}.rounded,.rounded-sm{border-radius:8px!important}.rounded-top{border-top-left-radius:8px!important}.rounded-right,.rounded-top{border-top-right-radius:8px!important}.rounded-bottom,.rounded-right{border-bottom-right-radius:8px!important}.rounded-bottom,.rounded-left{border-bottom-left-radius:8px!important}.rounded-left{border-top-left-radius:8px!important}.rounded-lg{border-radius:8px!important}.rounded-circle{border-radius:50%!important}.rounded-pill{border-radius:50rem!important}.rounded-0{border-radius:0!important}.clearfix:after{display:block;clear:both;content:""}.d-none{display:none!important}.d-inline{display:inline!important}.d-inline-block{display:inline-block!important}.d-block{display:block!important}.d-table{display:table!important}.d-table-row{display:table-row!important}.d-table-cell{display:table-cell!important}.d-flex{display:flex!important}.d-inline-flex{display:inline-flex!important}@media(min-width:400px){.d-xs-none{display:none!important}.d-xs-inline{display:inline!important}.d-xs-inline-block{display:inline-block!important}.d-xs-block{display:block!important}.d-xs-table{display:table!important}.d-xs-table-row{display:table-row!important}.d-xs-table-cell{display:table-cell!important}.d-xs-flex{display:flex!important}.d-xs-inline-flex{display:inline-flex!important}}@media(min-width:616px){.d-sm-none{display:none!important}.d-sm-inline{display:inline!important}.d-sm-inline-block{display:inline-block!important}.d-sm-block{display:block!important}.d-sm-table{display:table!important}.d-sm-table-row{display:table-row!important}.d-sm-table-cell{display:table-cell!important}.d-sm-flex{display:flex!important}.d-sm-inline-flex{display:inline-flex!important}}@media(min-width:768px){.d-md-none{display:none!important}.d-md-inline{display:inline!important}.d-md-inline-block{display:inline-block!important}.d-md-block{display:block!important}.d-md-table{display:table!important}.d-md-table-row{display:table-row!important}.d-md-table-cell{display:table-cell!important}.d-md-flex{display:flex!important}.d-md-inline-flex{display:inline-flex!important}}@media(min-width:980px){.d-lg-none{display:none!important}.d-lg-inline{display:inline!important}.d-lg-inline-block{display:inline-block!important}.d-lg-block{display:block!important}.d-lg-table{display:table!important}.d-lg-table-row{display:table-row!important}.d-lg-table-cell{display:table-cell!important}.d-lg-flex{display:flex!important}.d-lg-inline-flex{display:inline-flex!important}}@media(min-width:1240px){.d-xl-none{display:none!important}.d-xl-inline{display:inline!important}.d-xl-inline-block{display:inline-block!important}.d-xl-block{display:block!important}.d-xl-table{display:table!important}.d-xl-table-row{display:table-row!important}.d-xl-table-cell{display:table-cell!important}.d-xl-flex{display:flex!important}.d-xl-inline-flex{display:inline-flex!important}}@media print{.d-print-none{display:none!important}.d-print-inline{display:inline!important}.d-print-inline-block{display:inline-block!important}.d-print-block{display:block!important}.d-print-table{display:table!important}.d-print-table-row{display:table-row!important}.d-print-table-cell{display:table-cell!important}.d-print-flex{display:flex!important}.d-print-inline-flex{display:inline-flex!important}}.embed-responsive{position:relative;display:block;width:100%;padding:0;overflow:hidden}.embed-responsive:before{display:block;content:""}.embed-responsive .embed-responsive-item,.embed-responsive embed,.embed-responsive iframe,.embed-responsive object,.embed-responsive video{position:absolute;top:0;bottom:0;left:0;width:100%;height:100%;border:0}.embed-responsive-21by9:before{padding-top:42.8571428571%}.embed-responsive-16by9:before{padding-top:56.25%}.embed-responsive-4by3:before{padding-top:75%}.embed-responsive-1by1:before{padding-top:100%}.flex-row{flex-direction:row!important}.flex-column{flex-direction:column!important}.flex-row-reverse{flex-direction:row-reverse!important}.flex-column-reverse{flex-direction:column-reverse!important}.flex-wrap{flex-wrap:wrap!important}.flex-nowrap{flex-wrap:nowrap!important}.flex-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-fill{flex:1 1 auto!important}.flex-grow-0{flex-grow:0!important}.flex-grow-1{flex-grow:1!important}.flex-shrink-0{flex-shrink:0!important}.flex-shrink-1{flex-shrink:1!important}.justify-content-start{justify-content:flex-start!important}.justify-content-end{justify-content:flex-end!important}.justify-content-center{justify-content:center!important}.justify-content-between{justify-content:space-between!important}.justify-content-around{justify-content:space-around!important}.align-items-start{align-items:flex-start!important}.align-items-end{align-items:flex-end!important}.align-items-center{align-items:center!important}.align-items-baseline{align-items:baseline!important}.align-items-stretch{align-items:stretch!important}.align-content-start{align-content:flex-start!important}.align-content-end{align-content:flex-end!important}.align-content-center{align-content:center!important}.align-content-between{align-content:space-between!important}.align-content-around{align-content:space-around!important}.align-content-stretch{align-content:stretch!important}.align-self-auto{align-self:auto!important}.align-self-start{align-self:flex-start!important}.align-self-end{align-self:flex-end!important}.align-self-center{align-self:center!important}.align-self-baseline{align-self:baseline!important}.align-self-stretch{align-self:stretch!important}@media(min-width:400px){.flex-xs-row{flex-direction:row!important}.flex-xs-column{flex-direction:column!important}.flex-xs-row-reverse{flex-direction:row-reverse!important}.flex-xs-column-reverse{flex-direction:column-reverse!important}.flex-xs-wrap{flex-wrap:wrap!important}.flex-xs-nowrap{flex-wrap:nowrap!important}.flex-xs-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-xs-fill{flex:1 1 auto!important}.flex-xs-grow-0{flex-grow:0!important}.flex-xs-grow-1{flex-grow:1!important}.flex-xs-shrink-0{flex-shrink:0!important}.flex-xs-shrink-1{flex-shrink:1!important}.justify-content-xs-start{justify-content:flex-start!important}.justify-content-xs-end{justify-content:flex-end!important}.justify-content-xs-center{justify-content:center!important}.justify-content-xs-between{justify-content:space-between!important}.justify-content-xs-around{justify-content:space-around!important}.align-items-xs-start{align-items:flex-start!important}.align-items-xs-end{align-items:flex-end!important}.align-items-xs-center{align-items:center!important}.align-items-xs-baseline{align-items:baseline!important}.align-items-xs-stretch{align-items:stretch!important}.align-content-xs-start{align-content:flex-start!important}.align-content-xs-end{align-content:flex-end!important}.align-content-xs-center{align-content:center!important}.align-content-xs-between{align-content:space-between!important}.align-content-xs-around{align-content:space-around!important}.align-content-xs-stretch{align-content:stretch!important}.align-self-xs-auto{align-self:auto!important}.align-self-xs-start{align-self:flex-start!important}.align-self-xs-end{align-self:flex-end!important}.align-self-xs-center{align-self:center!important}.align-self-xs-baseline{align-self:baseline!important}.align-self-xs-stretch{align-self:stretch!important}}@media(min-width:616px){.flex-sm-row{flex-direction:row!important}.flex-sm-column{flex-direction:column!important}.flex-sm-row-reverse{flex-direction:row-reverse!important}.flex-sm-column-reverse{flex-direction:column-reverse!important}.flex-sm-wrap{flex-wrap:wrap!important}.flex-sm-nowrap{flex-wrap:nowrap!important}.flex-sm-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-sm-fill{flex:1 1 auto!important}.flex-sm-grow-0{flex-grow:0!important}.flex-sm-grow-1{flex-grow:1!important}.flex-sm-shrink-0{flex-shrink:0!important}.flex-sm-shrink-1{flex-shrink:1!important}.justify-content-sm-start{justify-content:flex-start!important}.justify-content-sm-end{justify-content:flex-end!important}.justify-content-sm-center{justify-content:center!important}.justify-content-sm-between{justify-content:space-between!important}.justify-content-sm-around{justify-content:space-around!important}.align-items-sm-start{align-items:flex-start!important}.align-items-sm-end{align-items:flex-end!important}.align-items-sm-center{align-items:center!important}.align-items-sm-baseline{align-items:baseline!important}.align-items-sm-stretch{align-items:stretch!important}.align-content-sm-start{align-content:flex-start!important}.align-content-sm-end{align-content:flex-end!important}.align-content-sm-center{align-content:center!important}.align-content-sm-between{align-content:space-between!important}.align-content-sm-around{align-content:space-around!important}.align-content-sm-stretch{align-content:stretch!important}.align-self-sm-auto{align-self:auto!important}.align-self-sm-start{align-self:flex-start!important}.align-self-sm-end{align-self:flex-end!important}.align-self-sm-center{align-self:center!important}.align-self-sm-baseline{align-self:baseline!important}.align-self-sm-stretch{align-self:stretch!important}}@media(min-width:768px){.flex-md-row{flex-direction:row!important}.flex-md-column{flex-direction:column!important}.flex-md-row-reverse{flex-direction:row-reverse!important}.flex-md-column-reverse{flex-direction:column-reverse!important}.flex-md-wrap{flex-wrap:wrap!important}.flex-md-nowrap{flex-wrap:nowrap!important}.flex-md-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-md-fill{flex:1 1 auto!important}.flex-md-grow-0{flex-grow:0!important}.flex-md-grow-1{flex-grow:1!important}.flex-md-shrink-0{flex-shrink:0!important}.flex-md-shrink-1{flex-shrink:1!important}.justify-content-md-start{justify-content:flex-start!important}.justify-content-md-end{justify-content:flex-end!important}.justify-content-md-center{justify-content:center!important}.justify-content-md-between{justify-content:space-between!important}.justify-content-md-around{justify-content:space-around!important}.align-items-md-start{align-items:flex-start!important}.align-items-md-end{align-items:flex-end!important}.align-items-md-center{align-items:center!important}.align-items-md-baseline{align-items:baseline!important}.align-items-md-stretch{align-items:stretch!important}.align-content-md-start{align-content:flex-start!important}.align-content-md-end{align-content:flex-end!important}.align-content-md-center{align-content:center!important}.align-content-md-between{align-content:space-between!important}.align-content-md-around{align-content:space-around!important}.align-content-md-stretch{align-content:stretch!important}.align-self-md-auto{align-self:auto!important}.align-self-md-start{align-self:flex-start!important}.align-self-md-end{align-self:flex-end!important}.align-self-md-center{align-self:center!important}.align-self-md-baseline{align-self:baseline!important}.align-self-md-stretch{align-self:stretch!important}}@media(min-width:980px){.flex-lg-row{flex-direction:row!important}.flex-lg-column{flex-direction:column!important}.flex-lg-row-reverse{flex-direction:row-reverse!important}.flex-lg-column-reverse{flex-direction:column-reverse!important}.flex-lg-wrap{flex-wrap:wrap!important}.flex-lg-nowrap{flex-wrap:nowrap!important}.flex-lg-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-lg-fill{flex:1 1 auto!important}.flex-lg-grow-0{flex-grow:0!important}.flex-lg-grow-1{flex-grow:1!important}.flex-lg-shrink-0{flex-shrink:0!important}.flex-lg-shrink-1{flex-shrink:1!important}.justify-content-lg-start{justify-content:flex-start!important}.justify-content-lg-end{justify-content:flex-end!important}.justify-content-lg-center{justify-content:center!important}.justify-content-lg-between{justify-content:space-between!important}.justify-content-lg-around{justify-content:space-around!important}.align-items-lg-start{align-items:flex-start!important}.align-items-lg-end{align-items:flex-end!important}.align-items-lg-center{align-items:center!important}.align-items-lg-baseline{align-items:baseline!important}.align-items-lg-stretch{align-items:stretch!important}.align-content-lg-start{align-content:flex-start!important}.align-content-lg-end{align-content:flex-end!important}.align-content-lg-center{align-content:center!important}.align-content-lg-between{align-content:space-between!important}.align-content-lg-around{align-content:space-around!important}.align-content-lg-stretch{align-content:stretch!important}.align-self-lg-auto{align-self:auto!important}.align-self-lg-start{align-self:flex-start!important}.align-self-lg-end{align-self:flex-end!important}.align-self-lg-center{align-self:center!important}.align-self-lg-baseline{align-self:baseline!important}.align-self-lg-stretch{align-self:stretch!important}}@media(min-width:1240px){.flex-xl-row{flex-direction:row!important}.flex-xl-column{flex-direction:column!important}.flex-xl-row-reverse{flex-direction:row-reverse!important}.flex-xl-column-reverse{flex-direction:column-reverse!important}.flex-xl-wrap{flex-wrap:wrap!important}.flex-xl-nowrap{flex-wrap:nowrap!important}.flex-xl-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-xl-fill{flex:1 1 auto!important}.flex-xl-grow-0{flex-grow:0!important}.flex-xl-grow-1{flex-grow:1!important}.flex-xl-shrink-0{flex-shrink:0!important}.flex-xl-shrink-1{flex-shrink:1!important}.justify-content-xl-start{justify-content:flex-start!important}.justify-content-xl-end{justify-content:flex-end!important}.justify-content-xl-center{justify-content:center!important}.justify-content-xl-between{justify-content:space-between!important}.justify-content-xl-around{justify-content:space-around!important}.align-items-xl-start{align-items:flex-start!important}.align-items-xl-end{align-items:flex-end!important}.align-items-xl-center{align-items:center!important}.align-items-xl-baseline{align-items:baseline!important}.align-items-xl-stretch{align-items:stretch!important}.align-content-xl-start{align-content:flex-start!important}.align-content-xl-end{align-content:flex-end!important}.align-content-xl-center{align-content:center!important}.align-content-xl-between{align-content:space-between!important}.align-content-xl-around{align-content:space-around!important}.align-content-xl-stretch{align-content:stretch!important}.align-self-xl-auto{align-self:auto!important}.align-self-xl-start{align-self:flex-start!important}.align-self-xl-end{align-self:flex-end!important}.align-self-xl-center{align-self:center!important}.align-self-xl-baseline{align-self:baseline!important}.align-self-xl-stretch{align-self:stretch!important}}.float-left{float:left!important}.float-right{float:right!important}.float-none{float:none!important}@media(min-width:400px){.float-xs-left{float:left!important}.float-xs-right{float:right!important}.float-xs-none{float:none!important}}@media(min-width:616px){.float-sm-left{float:left!important}.float-sm-right{float:right!important}.float-sm-none{float:none!important}}@media(min-width:768px){.float-md-left{float:left!important}.float-md-right{float:right!important}.float-md-none{float:none!important}}@media(min-width:980px){.float-lg-left{float:left!important}.float-lg-right{float:right!important}.float-lg-none{float:none!important}}@media(min-width:1240px){.float-xl-left{float:left!important}.float-xl-right{float:right!important}.float-xl-none{float:none!important}}.overflow-auto{overflow:auto!important}.overflow-hidden{overflow:hidden!important}.position-static{position:static!important}.position-relative{position:relative!important}.position-absolute{position:absolute!important}.position-fixed{position:fixed!important}.position-sticky{position:sticky!important}.fixed-top{top:0}.fixed-bottom,.fixed-top{position:fixed;right:0;left:0;z-index:1030}.fixed-bottom{bottom:0}@supports(position:sticky){.sticky-top{position:sticky;top:0;z-index:1020}}.sr-only{position:absolute;width:1px;height:1px;padding:0;margin:-1px;overflow:hidden;clip:rect(0,0,0,0);white-space:nowrap;border:0}.sr-only-focusable:active,.sr-only-focusable:focus{position:static;width:auto;height:auto;overflow:visible;clip:auto;white-space:normal}.shadow-sm{box-shadow:0 2px 14px rgba(108,117,125,.2)!important}.shadow{box-shadow:0 8px 20px rgba(108,117,125,.2)!important}.shadow-lg{box-shadow:0 12px 32px rgba(108,117,125,.2)!important}.shadow-none{box-shadow:none!important}.w-25{width:25%!important}.w-50{width:50%!important}.w-75{width:75%!important}.w-100{width:100%!important}.w-auto{width:auto!important}.h-25{height:25%!important}.h-50{height:50%!important}.h-75{height:75%!important}.h-100{height:100%!important}.h-auto{height:auto!important}.mw-100{max-width:100%!important}.mh-100{max-height:100%!important}.min-vw-100{min-width:100vw!important}.min-vh-100{min-height:100vh!important}.vw-100{width:100vw!important}.vh-100{height:100vh!important}.stretched-link:after{position:absolute;top:0;right:0;bottom:0;left:0;z-index:1;pointer-events:auto;content:"";background-color:transparent}.m-0{margin:0!important}.mt-0,.my-0{margin-top:0!important}.mr-0,.mx-0{margin-right:0!important}.mb-0,.my-0{margin-bottom:0!important}.ml-0,.mx-0{margin-left:0!important}.m-1{margin:8px!important}.mt-1,.my-1{margin-top:8px!important}.mr-1,.mx-1{margin-right:8px!important}.mb-1,.my-1{margin-bottom:8px!important}.ml-1,.mx-1{margin-left:8px!important}.m-2{margin:16px!important}.mt-2,.my-2{margin-top:16px!important}.mr-2,.mx-2{margin-right:16px!important}.mb-2,.my-2{margin-bottom:16px!important}.ml-2,.mx-2{margin-left:16px!important}.m-3{margin:24px!important}.mt-3,.my-3{margin-top:24px!important}.mr-3,.mx-3{margin-right:24px!important}.mb-3,.my-3{margin-bottom:24px!important}.ml-3,.mx-3{margin-left:24px!important}.m-4{margin:32px!important}.mt-4,.my-4{margin-top:32px!important}.mr-4,.mx-4{margin-right:32px!important}.mb-4,.my-4{margin-bottom:32px!important}.ml-4,.mx-4{margin-left:32px!important}.m-5{margin:40px!important}.mt-5,.my-5{margin-top:40px!important}.mr-5,.mx-5{margin-right:40px!important}.mb-5,.my-5{margin-bottom:40px!important}.ml-5,.mx-5{margin-left:40px!important}.m-6{margin:48px!important}.mt-6,.my-6{margin-top:48px!important}.mr-6,.mx-6{margin-right:48px!important}.mb-6,.my-6{margin-bottom:48px!important}.ml-6,.mx-6{margin-left:48px!important}.m-7{margin:56px!important}.mt-7,.my-7{margin-top:56px!important}.mr-7,.mx-7{margin-right:56px!important}.mb-7,.my-7{margin-bottom:56px!important}.ml-7,.mx-7{margin-left:56px!important}.m-8{margin:64px!important}.mt-8,.my-8{margin-top:64px!important}.mr-8,.mx-8{margin-right:64px!important}.mb-8,.my-8{margin-bottom:64px!important}.ml-8,.mx-8{margin-left:64px!important}.m-9{margin:72px!important}.mt-9,.my-9{margin-top:72px!important}.mr-9,.mx-9{margin-right:72px!important}.mb-9,.my-9{margin-bottom:72px!important}.ml-9,.mx-9{margin-left:72px!important}.m-10{margin:80px!important}.mt-10,.my-10{margin-top:80px!important}.mr-10,.mx-10{margin-right:80px!important}.mb-10,.my-10{margin-bottom:80px!important}.ml-10,.mx-10{margin-left:80px!important}.m-12{margin:96px!important}.mt-12,.my-12{margin-top:96px!important}.mr-12,.mx-12{margin-right:96px!important}.mb-12,.my-12{margin-bottom:96px!important}.ml-12,.mx-12{margin-left:96px!important}.m-15{margin:120px!important}.mt-15,.my-15{margin-top:120px!important}.mr-15,.mx-15{margin-right:120px!important}.mb-15,.my-15{margin-bottom:120px!important}.ml-15,.mx-15{margin-left:120px!important}.p-0{padding:0!important}.pt-0,.py-0{padding-top:0!important}.pr-0,.px-0{padding-right:0!important}.pb-0,.py-0{padding-bottom:0!important}.pl-0,.px-0{padding-left:0!important}.p-1{padding:8px!important}.pt-1,.py-1{padding-top:8px!important}.pr-1,.px-1{padding-right:8px!important}.pb-1,.py-1{padding-bottom:8px!important}.pl-1,.px-1{padding-left:8px!important}.p-2{padding:16px!important}.pt-2,.py-2{padding-top:16px!important}.pr-2,.px-2{padding-right:16px!important}.pb-2,.py-2{padding-bottom:16px!important}.pl-2,.px-2{padding-left:16px!important}.p-3{padding:24px!important}.pt-3,.py-3{padding-top:24px!important}.pr-3,.px-3{padding-right:24px!important}.pb-3,.py-3{padding-bottom:24px!important}.pl-3,.px-3{padding-left:24px!important}.p-4{padding:32px!important}.pt-4,.py-4{padding-top:32px!important}.pr-4,.px-4{padding-right:32px!important}.pb-4,.py-4{padding-bottom:32px!important}.pl-4,.px-4{padding-left:32px!important}.p-5{padding:40px!important}.pt-5,.py-5{padding-top:40px!important}.pr-5,.px-5{padding-right:40px!important}.pb-5,.py-5{padding-bottom:40px!important}.pl-5,.px-5{padding-left:40px!important}.p-6{padding:48px!important}.pt-6,.py-6{padding-top:48px!important}.pr-6,.px-6{padding-right:48px!important}.pb-6,.py-6{padding-bottom:48px!important}.pl-6,.px-6{padding-left:48px!important}.p-7{padding:56px!important}.pt-7,.py-7{padding-top:56px!important}.pr-7,.px-7{padding-right:56px!important}.pb-7,.py-7{padding-bottom:56px!important}.pl-7,.px-7{padding-left:56px!important}.p-8{padding:64px!important}.pt-8,.py-8{padding-top:64px!important}.pr-8,.px-8{padding-right:64px!important}.pb-8,.py-8{padding-bottom:64px!important}.pl-8,.px-8{padding-left:64px!important}.p-9{padding:72px!important}.pt-9,.py-9{padding-top:72px!important}.pr-9,.px-9{padding-right:72px!important}.pb-9,.py-9{padding-bottom:72px!important}.pl-9,.px-9{padding-left:72px!important}.p-10{padding:80px!important}.pt-10,.py-10{padding-top:80px!important}.pr-10,.px-10{padding-right:80px!important}.pb-10,.py-10{padding-bottom:80px!important}.pl-10,.px-10{padding-left:80px!important}.p-12{padding:96px!important}.pt-12,.py-12{padding-top:96px!important}.pr-12,.px-12{padding-right:96px!important}.pb-12,.py-12{padding-bottom:96px!important}.pl-12,.px-12{padding-left:96px!important}.p-15{padding:120px!important}.pt-15,.py-15{padding-top:120px!important}.pr-15,.px-15{padding-right:120px!important}.pb-15,.py-15{padding-bottom:120px!important}.pl-15,.px-15{padding-left:120px!important}.m-n1{margin:-8px!important}.mt-n1,.my-n1{margin-top:-8px!important}.mr-n1,.mx-n1{margin-right:-8px!important}.mb-n1,.my-n1{margin-bottom:-8px!important}.ml-n1,.mx-n1{margin-left:-8px!important}.m-n2{margin:-16px!important}.mt-n2,.my-n2{margin-top:-16px!important}.mr-n2,.mx-n2{margin-right:-16px!important}.mb-n2,.my-n2{margin-bottom:-16px!important}.ml-n2,.mx-n2{margin-left:-16px!important}.m-n3{margin:-24px!important}.mt-n3,.my-n3{margin-top:-24px!important}.mr-n3,.mx-n3{margin-right:-24px!important}.mb-n3,.my-n3{margin-bottom:-24px!important}.ml-n3,.mx-n3{margin-left:-24px!important}.m-n4{margin:-32px!important}.mt-n4,.my-n4{margin-top:-32px!important}.mr-n4,.mx-n4{margin-right:-32px!important}.mb-n4,.my-n4{margin-bottom:-32px!important}.ml-n4,.mx-n4{margin-left:-32px!important}.m-n5{margin:-40px!important}.mt-n5,.my-n5{margin-top:-40px!important}.mr-n5,.mx-n5{margin-right:-40px!important}.mb-n5,.my-n5{margin-bottom:-40px!important}.ml-n5,.mx-n5{margin-left:-40px!important}.m-n6{margin:-48px!important}.mt-n6,.my-n6{margin-top:-48px!important}.mr-n6,.mx-n6{margin-right:-48px!important}.mb-n6,.my-n6{margin-bottom:-48px!important}.ml-n6,.mx-n6{margin-left:-48px!important}.m-n7{margin:-56px!important}.mt-n7,.my-n7{margin-top:-56px!important}.mr-n7,.mx-n7{margin-right:-56px!important}.mb-n7,.my-n7{margin-bottom:-56px!important}.ml-n7,.mx-n7{margin-left:-56px!important}.m-n8{margin:-64px!important}.mt-n8,.my-n8{margin-top:-64px!important}.mr-n8,.mx-n8{margin-right:-64px!important}.mb-n8,.my-n8{margin-bottom:-64px!important}.ml-n8,.mx-n8{margin-left:-64px!important}.m-n9{margin:-72px!important}.mt-n9,.my-n9{margin-top:-72px!important}.mr-n9,.mx-n9{margin-right:-72px!important}.mb-n9,.my-n9{margin-bottom:-72px!important}.ml-n9,.mx-n9{margin-left:-72px!important}.m-n10{margin:-80px!important}.mt-n10,.my-n10{margin-top:-80px!important}.mr-n10,.mx-n10{margin-right:-80px!important}.mb-n10,.my-n10{margin-bottom:-80px!important}.ml-n10,.mx-n10{margin-left:-80px!important}.m-n12{margin:-96px!important}.mt-n12,.my-n12{margin-top:-96px!important}.mr-n12,.mx-n12{margin-right:-96px!important}.mb-n12,.my-n12{margin-bottom:-96px!important}.ml-n12,.mx-n12{margin-left:-96px!important}.m-n15{margin:-120px!important}.mt-n15,.my-n15{margin-top:-120px!important}.mr-n15,.mx-n15{margin-right:-120px!important}.mb-n15,.my-n15{margin-bottom:-120px!important}.ml-n15,.mx-n15{margin-left:-120px!important}.m-auto{margin:auto!important}.mt-auto,.my-auto{margin-top:auto!important}.mr-auto,.mx-auto{margin-right:auto!important}.mb-auto,.my-auto{margin-bottom:auto!important}.ml-auto,.mx-auto{margin-left:auto!important}@media(min-width:400px){.m-xs-0{margin:0!important}.mt-xs-0,.my-xs-0{margin-top:0!important}.mr-xs-0,.mx-xs-0{margin-right:0!important}.mb-xs-0,.my-xs-0{margin-bottom:0!important}.ml-xs-0,.mx-xs-0{margin-left:0!important}.m-xs-1{margin:8px!important}.mt-xs-1,.my-xs-1{margin-top:8px!important}.mr-xs-1,.mx-xs-1{margin-right:8px!important}.mb-xs-1,.my-xs-1{margin-bottom:8px!important}.ml-xs-1,.mx-xs-1{margin-left:8px!important}.m-xs-2{margin:16px!important}.mt-xs-2,.my-xs-2{margin-top:16px!important}.mr-xs-2,.mx-xs-2{margin-right:16px!important}.mb-xs-2,.my-xs-2{margin-bottom:16px!important}.ml-xs-2,.mx-xs-2{margin-left:16px!important}.m-xs-3{margin:24px!important}.mt-xs-3,.my-xs-3{margin-top:24px!important}.mr-xs-3,.mx-xs-3{margin-right:24px!important}.mb-xs-3,.my-xs-3{margin-bottom:24px!important}.ml-xs-3,.mx-xs-3{margin-left:24px!important}.m-xs-4{margin:32px!important}.mt-xs-4,.my-xs-4{margin-top:32px!important}.mr-xs-4,.mx-xs-4{margin-right:32px!important}.mb-xs-4,.my-xs-4{margin-bottom:32px!important}.ml-xs-4,.mx-xs-4{margin-left:32px!important}.m-xs-5{margin:40px!important}.mt-xs-5,.my-xs-5{margin-top:40px!important}.mr-xs-5,.mx-xs-5{margin-right:40px!important}.mb-xs-5,.my-xs-5{margin-bottom:40px!important}.ml-xs-5,.mx-xs-5{margin-left:40px!important}.m-xs-6{margin:48px!important}.mt-xs-6,.my-xs-6{margin-top:48px!important}.mr-xs-6,.mx-xs-6{margin-right:48px!important}.mb-xs-6,.my-xs-6{margin-bottom:48px!important}.ml-xs-6,.mx-xs-6{margin-left:48px!important}.m-xs-7{margin:56px!important}.mt-xs-7,.my-xs-7{margin-top:56px!important}.mr-xs-7,.mx-xs-7{margin-right:56px!important}.mb-xs-7,.my-xs-7{margin-bottom:56px!important}.ml-xs-7,.mx-xs-7{margin-left:56px!important}.m-xs-8{margin:64px!important}.mt-xs-8,.my-xs-8{margin-top:64px!important}.mr-xs-8,.mx-xs-8{margin-right:64px!important}.mb-xs-8,.my-xs-8{margin-bottom:64px!important}.ml-xs-8,.mx-xs-8{margin-left:64px!important}.m-xs-9{margin:72px!important}.mt-xs-9,.my-xs-9{margin-top:72px!important}.mr-xs-9,.mx-xs-9{margin-right:72px!important}.mb-xs-9,.my-xs-9{margin-bottom:72px!important}.ml-xs-9,.mx-xs-9{margin-left:72px!important}.m-xs-10{margin:80px!important}.mt-xs-10,.my-xs-10{margin-top:80px!important}.mr-xs-10,.mx-xs-10{margin-right:80px!important}.mb-xs-10,.my-xs-10{margin-bottom:80px!important}.ml-xs-10,.mx-xs-10{margin-left:80px!important}.m-xs-12{margin:96px!important}.mt-xs-12,.my-xs-12{margin-top:96px!important}.mr-xs-12,.mx-xs-12{margin-right:96px!important}.mb-xs-12,.my-xs-12{margin-bottom:96px!important}.ml-xs-12,.mx-xs-12{margin-left:96px!important}.m-xs-15{margin:120px!important}.mt-xs-15,.my-xs-15{margin-top:120px!important}.mr-xs-15,.mx-xs-15{margin-right:120px!important}.mb-xs-15,.my-xs-15{margin-bottom:120px!important}.ml-xs-15,.mx-xs-15{margin-left:120px!important}.p-xs-0{padding:0!important}.pt-xs-0,.py-xs-0{padding-top:0!important}.pr-xs-0,.px-xs-0{padding-right:0!important}.pb-xs-0,.py-xs-0{padding-bottom:0!important}.pl-xs-0,.px-xs-0{padding-left:0!important}.p-xs-1{padding:8px!important}.pt-xs-1,.py-xs-1{padding-top:8px!important}.pr-xs-1,.px-xs-1{padding-right:8px!important}.pb-xs-1,.py-xs-1{padding-bottom:8px!important}.pl-xs-1,.px-xs-1{padding-left:8px!important}.p-xs-2{padding:16px!important}.pt-xs-2,.py-xs-2{padding-top:16px!important}.pr-xs-2,.px-xs-2{padding-right:16px!important}.pb-xs-2,.py-xs-2{padding-bottom:16px!important}.pl-xs-2,.px-xs-2{padding-left:16px!important}.p-xs-3{padding:24px!important}.pt-xs-3,.py-xs-3{padding-top:24px!important}.pr-xs-3,.px-xs-3{padding-right:24px!important}.pb-xs-3,.py-xs-3{padding-bottom:24px!important}.pl-xs-3,.px-xs-3{padding-left:24px!important}.p-xs-4{padding:32px!important}.pt-xs-4,.py-xs-4{padding-top:32px!important}.pr-xs-4,.px-xs-4{padding-right:32px!important}.pb-xs-4,.py-xs-4{padding-bottom:32px!important}.pl-xs-4,.px-xs-4{padding-left:32px!important}.p-xs-5{padding:40px!important}.pt-xs-5,.py-xs-5{padding-top:40px!important}.pr-xs-5,.px-xs-5{padding-right:40px!important}.pb-xs-5,.py-xs-5{padding-bottom:40px!important}.pl-xs-5,.px-xs-5{padding-left:40px!important}.p-xs-6{padding:48px!important}.pt-xs-6,.py-xs-6{padding-top:48px!important}.pr-xs-6,.px-xs-6{padding-right:48px!important}.pb-xs-6,.py-xs-6{padding-bottom:48px!important}.pl-xs-6,.px-xs-6{padding-left:48px!important}.p-xs-7{padding:56px!important}.pt-xs-7,.py-xs-7{padding-top:56px!important}.pr-xs-7,.px-xs-7{padding-right:56px!important}.pb-xs-7,.py-xs-7{padding-bottom:56px!important}.pl-xs-7,.px-xs-7{padding-left:56px!important}.p-xs-8{padding:64px!important}.pt-xs-8,.py-xs-8{padding-top:64px!important}.pr-xs-8,.px-xs-8{padding-right:64px!important}.pb-xs-8,.py-xs-8{padding-bottom:64px!important}.pl-xs-8,.px-xs-8{padding-left:64px!important}.p-xs-9{padding:72px!important}.pt-xs-9,.py-xs-9{padding-top:72px!important}.pr-xs-9,.px-xs-9{padding-right:72px!important}.pb-xs-9,.py-xs-9{padding-bottom:72px!important}.pl-xs-9,.px-xs-9{padding-left:72px!important}.p-xs-10{padding:80px!important}.pt-xs-10,.py-xs-10{padding-top:80px!important}.pr-xs-10,.px-xs-10{padding-right:80px!important}.pb-xs-10,.py-xs-10{padding-bottom:80px!important}.pl-xs-10,.px-xs-10{padding-left:80px!important}.p-xs-12{padding:96px!important}.pt-xs-12,.py-xs-12{padding-top:96px!important}.pr-xs-12,.px-xs-12{padding-right:96px!important}.pb-xs-12,.py-xs-12{padding-bottom:96px!important}.pl-xs-12,.px-xs-12{padding-left:96px!important}.p-xs-15{padding:120px!important}.pt-xs-15,.py-xs-15{padding-top:120px!important}.pr-xs-15,.px-xs-15{padding-right:120px!important}.pb-xs-15,.py-xs-15{padding-bottom:120px!important}.pl-xs-15,.px-xs-15{padding-left:120px!important}.m-xs-n1{margin:-8px!important}.mt-xs-n1,.my-xs-n1{margin-top:-8px!important}.mr-xs-n1,.mx-xs-n1{margin-right:-8px!important}.mb-xs-n1,.my-xs-n1{margin-bottom:-8px!important}.ml-xs-n1,.mx-xs-n1{margin-left:-8px!important}.m-xs-n2{margin:-16px!important}.mt-xs-n2,.my-xs-n2{margin-top:-16px!important}.mr-xs-n2,.mx-xs-n2{margin-right:-16px!important}.mb-xs-n2,.my-xs-n2{margin-bottom:-16px!important}.ml-xs-n2,.mx-xs-n2{margin-left:-16px!important}.m-xs-n3{margin:-24px!important}.mt-xs-n3,.my-xs-n3{margin-top:-24px!important}.mr-xs-n3,.mx-xs-n3{margin-right:-24px!important}.mb-xs-n3,.my-xs-n3{margin-bottom:-24px!important}.ml-xs-n3,.mx-xs-n3{margin-left:-24px!important}.m-xs-n4{margin:-32px!important}.mt-xs-n4,.my-xs-n4{margin-top:-32px!important}.mr-xs-n4,.mx-xs-n4{margin-right:-32px!important}.mb-xs-n4,.my-xs-n4{margin-bottom:-32px!important}.ml-xs-n4,.mx-xs-n4{margin-left:-32px!important}.m-xs-n5{margin:-40px!important}.mt-xs-n5,.my-xs-n5{margin-top:-40px!important}.mr-xs-n5,.mx-xs-n5{margin-right:-40px!important}.mb-xs-n5,.my-xs-n5{margin-bottom:-40px!important}.ml-xs-n5,.mx-xs-n5{margin-left:-40px!important}.m-xs-n6{margin:-48px!important}.mt-xs-n6,.my-xs-n6{margin-top:-48px!important}.mr-xs-n6,.mx-xs-n6{margin-right:-48px!important}.mb-xs-n6,.my-xs-n6{margin-bottom:-48px!important}.ml-xs-n6,.mx-xs-n6{margin-left:-48px!important}.m-xs-n7{margin:-56px!important}.mt-xs-n7,.my-xs-n7{margin-top:-56px!important}.mr-xs-n7,.mx-xs-n7{margin-right:-56px!important}.mb-xs-n7,.my-xs-n7{margin-bottom:-56px!important}.ml-xs-n7,.mx-xs-n7{margin-left:-56px!important}.m-xs-n8{margin:-64px!important}.mt-xs-n8,.my-xs-n8{margin-top:-64px!important}.mr-xs-n8,.mx-xs-n8{margin-right:-64px!important}.mb-xs-n8,.my-xs-n8{margin-bottom:-64px!important}.ml-xs-n8,.mx-xs-n8{margin-left:-64px!important}.m-xs-n9{margin:-72px!important}.mt-xs-n9,.my-xs-n9{margin-top:-72px!important}.mr-xs-n9,.mx-xs-n9{margin-right:-72px!important}.mb-xs-n9,.my-xs-n9{margin-bottom:-72px!important}.ml-xs-n9,.mx-xs-n9{margin-left:-72px!important}.m-xs-n10{margin:-80px!important}.mt-xs-n10,.my-xs-n10{margin-top:-80px!important}.mr-xs-n10,.mx-xs-n10{margin-right:-80px!important}.mb-xs-n10,.my-xs-n10{margin-bottom:-80px!important}.ml-xs-n10,.mx-xs-n10{margin-left:-80px!important}.m-xs-n12{margin:-96px!important}.mt-xs-n12,.my-xs-n12{margin-top:-96px!important}.mr-xs-n12,.mx-xs-n12{margin-right:-96px!important}.mb-xs-n12,.my-xs-n12{margin-bottom:-96px!important}.ml-xs-n12,.mx-xs-n12{margin-left:-96px!important}.m-xs-n15{margin:-120px!important}.mt-xs-n15,.my-xs-n15{margin-top:-120px!important}.mr-xs-n15,.mx-xs-n15{margin-right:-120px!important}.mb-xs-n15,.my-xs-n15{margin-bottom:-120px!important}.ml-xs-n15,.mx-xs-n15{margin-left:-120px!important}.m-xs-auto{margin:auto!important}.mt-xs-auto,.my-xs-auto{margin-top:auto!important}.mr-xs-auto,.mx-xs-auto{margin-right:auto!important}.mb-xs-auto,.my-xs-auto{margin-bottom:auto!important}.ml-xs-auto,.mx-xs-auto{margin-left:auto!important}}@media(min-width:616px){.m-sm-0{margin:0!important}.mt-sm-0,.my-sm-0{margin-top:0!important}.mr-sm-0,.mx-sm-0{margin-right:0!important}.mb-sm-0,.my-sm-0{margin-bottom:0!important}.ml-sm-0,.mx-sm-0{margin-left:0!important}.m-sm-1{margin:8px!important}.mt-sm-1,.my-sm-1{margin-top:8px!important}.mr-sm-1,.mx-sm-1{margin-right:8px!important}.mb-sm-1,.my-sm-1{margin-bottom:8px!important}.ml-sm-1,.mx-sm-1{margin-left:8px!important}.m-sm-2{margin:16px!important}.mt-sm-2,.my-sm-2{margin-top:16px!important}.mr-sm-2,.mx-sm-2{margin-right:16px!important}.mb-sm-2,.my-sm-2{margin-bottom:16px!important}.ml-sm-2,.mx-sm-2{margin-left:16px!important}.m-sm-3{margin:24px!important}.mt-sm-3,.my-sm-3{margin-top:24px!important}.mr-sm-3,.mx-sm-3{margin-right:24px!important}.mb-sm-3,.my-sm-3{margin-bottom:24px!important}.ml-sm-3,.mx-sm-3{margin-left:24px!important}.m-sm-4{margin:32px!important}.mt-sm-4,.my-sm-4{margin-top:32px!important}.mr-sm-4,.mx-sm-4{margin-right:32px!important}.mb-sm-4,.my-sm-4{margin-bottom:32px!important}.ml-sm-4,.mx-sm-4{margin-left:32px!important}.m-sm-5{margin:40px!important}.mt-sm-5,.my-sm-5{margin-top:40px!important}.mr-sm-5,.mx-sm-5{margin-right:40px!important}.mb-sm-5,.my-sm-5{margin-bottom:40px!important}.ml-sm-5,.mx-sm-5{margin-left:40px!important}.m-sm-6{margin:48px!important}.mt-sm-6,.my-sm-6{margin-top:48px!important}.mr-sm-6,.mx-sm-6{margin-right:48px!important}.mb-sm-6,.my-sm-6{margin-bottom:48px!important}.ml-sm-6,.mx-sm-6{margin-left:48px!important}.m-sm-7{margin:56px!important}.mt-sm-7,.my-sm-7{margin-top:56px!important}.mr-sm-7,.mx-sm-7{margin-right:56px!important}.mb-sm-7,.my-sm-7{margin-bottom:56px!important}.ml-sm-7,.mx-sm-7{margin-left:56px!important}.m-sm-8{margin:64px!important}.mt-sm-8,.my-sm-8{margin-top:64px!important}.mr-sm-8,.mx-sm-8{margin-right:64px!important}.mb-sm-8,.my-sm-8{margin-bottom:64px!important}.ml-sm-8,.mx-sm-8{margin-left:64px!important}.m-sm-9{margin:72px!important}.mt-sm-9,.my-sm-9{margin-top:72px!important}.mr-sm-9,.mx-sm-9{margin-right:72px!important}.mb-sm-9,.my-sm-9{margin-bottom:72px!important}.ml-sm-9,.mx-sm-9{margin-left:72px!important}.m-sm-10{margin:80px!important}.mt-sm-10,.my-sm-10{margin-top:80px!important}.mr-sm-10,.mx-sm-10{margin-right:80px!important}.mb-sm-10,.my-sm-10{margin-bottom:80px!important}.ml-sm-10,.mx-sm-10{margin-left:80px!important}.m-sm-12{margin:96px!important}.mt-sm-12,.my-sm-12{margin-top:96px!important}.mr-sm-12,.mx-sm-12{margin-right:96px!important}.mb-sm-12,.my-sm-12{margin-bottom:96px!important}.ml-sm-12,.mx-sm-12{margin-left:96px!important}.m-sm-15{margin:120px!important}.mt-sm-15,.my-sm-15{margin-top:120px!important}.mr-sm-15,.mx-sm-15{margin-right:120px!important}.mb-sm-15,.my-sm-15{margin-bottom:120px!important}.ml-sm-15,.mx-sm-15{margin-left:120px!important}.p-sm-0{padding:0!important}.pt-sm-0,.py-sm-0{padding-top:0!important}.pr-sm-0,.px-sm-0{padding-right:0!important}.pb-sm-0,.py-sm-0{padding-bottom:0!important}.pl-sm-0,.px-sm-0{padding-left:0!important}.p-sm-1{padding:8px!important}.pt-sm-1,.py-sm-1{padding-top:8px!important}.pr-sm-1,.px-sm-1{padding-right:8px!important}.pb-sm-1,.py-sm-1{padding-bottom:8px!important}.pl-sm-1,.px-sm-1{padding-left:8px!important}.p-sm-2{padding:16px!important}.pt-sm-2,.py-sm-2{padding-top:16px!important}.pr-sm-2,.px-sm-2{padding-right:16px!important}.pb-sm-2,.py-sm-2{padding-bottom:16px!important}.pl-sm-2,.px-sm-2{padding-left:16px!important}.p-sm-3{padding:24px!important}.pt-sm-3,.py-sm-3{padding-top:24px!important}.pr-sm-3,.px-sm-3{padding-right:24px!important}.pb-sm-3,.py-sm-3{padding-bottom:24px!important}.pl-sm-3,.px-sm-3{padding-left:24px!important}.p-sm-4{padding:32px!important}.pt-sm-4,.py-sm-4{padding-top:32px!important}.pr-sm-4,.px-sm-4{padding-right:32px!important}.pb-sm-4,.py-sm-4{padding-bottom:32px!important}.pl-sm-4,.px-sm-4{padding-left:32px!important}.p-sm-5{padding:40px!important}.pt-sm-5,.py-sm-5{padding-top:40px!important}.pr-sm-5,.px-sm-5{padding-right:40px!important}.pb-sm-5,.py-sm-5{padding-bottom:40px!important}.pl-sm-5,.px-sm-5{padding-left:40px!important}.p-sm-6{padding:48px!important}.pt-sm-6,.py-sm-6{padding-top:48px!important}.pr-sm-6,.px-sm-6{padding-right:48px!important}.pb-sm-6,.py-sm-6{padding-bottom:48px!important}.pl-sm-6,.px-sm-6{padding-left:48px!important}.p-sm-7{padding:56px!important}.pt-sm-7,.py-sm-7{padding-top:56px!important}.pr-sm-7,.px-sm-7{padding-right:56px!important}.pb-sm-7,.py-sm-7{padding-bottom:56px!important}.pl-sm-7,.px-sm-7{padding-left:56px!important}.p-sm-8{padding:64px!important}.pt-sm-8,.py-sm-8{padding-top:64px!important}.pr-sm-8,.px-sm-8{padding-right:64px!important}.pb-sm-8,.py-sm-8{padding-bottom:64px!important}.pl-sm-8,.px-sm-8{padding-left:64px!important}.p-sm-9{padding:72px!important}.pt-sm-9,.py-sm-9{padding-top:72px!important}.pr-sm-9,.px-sm-9{padding-right:72px!important}.pb-sm-9,.py-sm-9{padding-bottom:72px!important}.pl-sm-9,.px-sm-9{padding-left:72px!important}.p-sm-10{padding:80px!important}.pt-sm-10,.py-sm-10{padding-top:80px!important}.pr-sm-10,.px-sm-10{padding-right:80px!important}.pb-sm-10,.py-sm-10{padding-bottom:80px!important}.pl-sm-10,.px-sm-10{padding-left:80px!important}.p-sm-12{padding:96px!important}.pt-sm-12,.py-sm-12{padding-top:96px!important}.pr-sm-12,.px-sm-12{padding-right:96px!important}.pb-sm-12,.py-sm-12{padding-bottom:96px!important}.pl-sm-12,.px-sm-12{padding-left:96px!important}.p-sm-15{padding:120px!important}.pt-sm-15,.py-sm-15{padding-top:120px!important}.pr-sm-15,.px-sm-15{padding-right:120px!important}.pb-sm-15,.py-sm-15{padding-bottom:120px!important}.pl-sm-15,.px-sm-15{padding-left:120px!important}.m-sm-n1{margin:-8px!important}.mt-sm-n1,.my-sm-n1{margin-top:-8px!important}.mr-sm-n1,.mx-sm-n1{margin-right:-8px!important}.mb-sm-n1,.my-sm-n1{margin-bottom:-8px!important}.ml-sm-n1,.mx-sm-n1{margin-left:-8px!important}.m-sm-n2{margin:-16px!important}.mt-sm-n2,.my-sm-n2{margin-top:-16px!important}.mr-sm-n2,.mx-sm-n2{margin-right:-16px!important}.mb-sm-n2,.my-sm-n2{margin-bottom:-16px!important}.ml-sm-n2,.mx-sm-n2{margin-left:-16px!important}.m-sm-n3{margin:-24px!important}.mt-sm-n3,.my-sm-n3{margin-top:-24px!important}.mr-sm-n3,.mx-sm-n3{margin-right:-24px!important}.mb-sm-n3,.my-sm-n3{margin-bottom:-24px!important}.ml-sm-n3,.mx-sm-n3{margin-left:-24px!important}.m-sm-n4{margin:-32px!important}.mt-sm-n4,.my-sm-n4{margin-top:-32px!important}.mr-sm-n4,.mx-sm-n4{margin-right:-32px!important}.mb-sm-n4,.my-sm-n4{margin-bottom:-32px!important}.ml-sm-n4,.mx-sm-n4{margin-left:-32px!important}.m-sm-n5{margin:-40px!important}.mt-sm-n5,.my-sm-n5{margin-top:-40px!important}.mr-sm-n5,.mx-sm-n5{margin-right:-40px!important}.mb-sm-n5,.my-sm-n5{margin-bottom:-40px!important}.ml-sm-n5,.mx-sm-n5{margin-left:-40px!important}.m-sm-n6{margin:-48px!important}.mt-sm-n6,.my-sm-n6{margin-top:-48px!important}.mr-sm-n6,.mx-sm-n6{margin-right:-48px!important}.mb-sm-n6,.my-sm-n6{margin-bottom:-48px!important}.ml-sm-n6,.mx-sm-n6{margin-left:-48px!important}.m-sm-n7{margin:-56px!important}.mt-sm-n7,.my-sm-n7{margin-top:-56px!important}.mr-sm-n7,.mx-sm-n7{margin-right:-56px!important}.mb-sm-n7,.my-sm-n7{margin-bottom:-56px!important}.ml-sm-n7,.mx-sm-n7{margin-left:-56px!important}.m-sm-n8{margin:-64px!important}.mt-sm-n8,.my-sm-n8{margin-top:-64px!important}.mr-sm-n8,.mx-sm-n8{margin-right:-64px!important}.mb-sm-n8,.my-sm-n8{margin-bottom:-64px!important}.ml-sm-n8,.mx-sm-n8{margin-left:-64px!important}.m-sm-n9{margin:-72px!important}.mt-sm-n9,.my-sm-n9{margin-top:-72px!important}.mr-sm-n9,.mx-sm-n9{margin-right:-72px!important}.mb-sm-n9,.my-sm-n9{margin-bottom:-72px!important}.ml-sm-n9,.mx-sm-n9{margin-left:-72px!important}.m-sm-n10{margin:-80px!important}.mt-sm-n10,.my-sm-n10{margin-top:-80px!important}.mr-sm-n10,.mx-sm-n10{margin-right:-80px!important}.mb-sm-n10,.my-sm-n10{margin-bottom:-80px!important}.ml-sm-n10,.mx-sm-n10{margin-left:-80px!important}.m-sm-n12{margin:-96px!important}.mt-sm-n12,.my-sm-n12{margin-top:-96px!important}.mr-sm-n12,.mx-sm-n12{margin-right:-96px!important}.mb-sm-n12,.my-sm-n12{margin-bottom:-96px!important}.ml-sm-n12,.mx-sm-n12{margin-left:-96px!important}.m-sm-n15{margin:-120px!important}.mt-sm-n15,.my-sm-n15{margin-top:-120px!important}.mr-sm-n15,.mx-sm-n15{margin-right:-120px!important}.mb-sm-n15,.my-sm-n15{margin-bottom:-120px!important}.ml-sm-n15,.mx-sm-n15{margin-left:-120px!important}.m-sm-auto{margin:auto!important}.mt-sm-auto,.my-sm-auto{margin-top:auto!important}.mr-sm-auto,.mx-sm-auto{margin-right:auto!important}.mb-sm-auto,.my-sm-auto{margin-bottom:auto!important}.ml-sm-auto,.mx-sm-auto{margin-left:auto!important}}@media(min-width:768px){.m-md-0{margin:0!important}.mt-md-0,.my-md-0{margin-top:0!important}.mr-md-0,.mx-md-0{margin-right:0!important}.mb-md-0,.my-md-0{margin-bottom:0!important}.ml-md-0,.mx-md-0{margin-left:0!important}.m-md-1{margin:8px!important}.mt-md-1,.my-md-1{margin-top:8px!important}.mr-md-1,.mx-md-1{margin-right:8px!important}.mb-md-1,.my-md-1{margin-bottom:8px!important}.ml-md-1,.mx-md-1{margin-left:8px!important}.m-md-2{margin:16px!important}.mt-md-2,.my-md-2{margin-top:16px!important}.mr-md-2,.mx-md-2{margin-right:16px!important}.mb-md-2,.my-md-2{margin-bottom:16px!important}.ml-md-2,.mx-md-2{margin-left:16px!important}.m-md-3{margin:24px!important}.mt-md-3,.my-md-3{margin-top:24px!important}.mr-md-3,.mx-md-3{margin-right:24px!important}.mb-md-3,.my-md-3{margin-bottom:24px!important}.ml-md-3,.mx-md-3{margin-left:24px!important}.m-md-4{margin:32px!important}.mt-md-4,.my-md-4{margin-top:32px!important}.mr-md-4,.mx-md-4{margin-right:32px!important}.mb-md-4,.my-md-4{margin-bottom:32px!important}.ml-md-4,.mx-md-4{margin-left:32px!important}.m-md-5{margin:40px!important}.mt-md-5,.my-md-5{margin-top:40px!important}.mr-md-5,.mx-md-5{margin-right:40px!important}.mb-md-5,.my-md-5{margin-bottom:40px!important}.ml-md-5,.mx-md-5{margin-left:40px!important}.m-md-6{margin:48px!important}.mt-md-6,.my-md-6{margin-top:48px!important}.mr-md-6,.mx-md-6{margin-right:48px!important}.mb-md-6,.my-md-6{margin-bottom:48px!important}.ml-md-6,.mx-md-6{margin-left:48px!important}.m-md-7{margin:56px!important}.mt-md-7,.my-md-7{margin-top:56px!important}.mr-md-7,.mx-md-7{margin-right:56px!important}.mb-md-7,.my-md-7{margin-bottom:56px!important}.ml-md-7,.mx-md-7{margin-left:56px!important}.m-md-8{margin:64px!important}.mt-md-8,.my-md-8{margin-top:64px!important}.mr-md-8,.mx-md-8{margin-right:64px!important}.mb-md-8,.my-md-8{margin-bottom:64px!important}.ml-md-8,.mx-md-8{margin-left:64px!important}.m-md-9{margin:72px!important}.mt-md-9,.my-md-9{margin-top:72px!important}.mr-md-9,.mx-md-9{margin-right:72px!important}.mb-md-9,.my-md-9{margin-bottom:72px!important}.ml-md-9,.mx-md-9{margin-left:72px!important}.m-md-10{margin:80px!important}.mt-md-10,.my-md-10{margin-top:80px!important}.mr-md-10,.mx-md-10{margin-right:80px!important}.mb-md-10,.my-md-10{margin-bottom:80px!important}.ml-md-10,.mx-md-10{margin-left:80px!important}.m-md-12{margin:96px!important}.mt-md-12,.my-md-12{margin-top:96px!important}.mr-md-12,.mx-md-12{margin-right:96px!important}.mb-md-12,.my-md-12{margin-bottom:96px!important}.ml-md-12,.mx-md-12{margin-left:96px!important}.m-md-15{margin:120px!important}.mt-md-15,.my-md-15{margin-top:120px!important}.mr-md-15,.mx-md-15{margin-right:120px!important}.mb-md-15,.my-md-15{margin-bottom:120px!important}.ml-md-15,.mx-md-15{margin-left:120px!important}.p-md-0{padding:0!important}.pt-md-0,.py-md-0{padding-top:0!important}.pr-md-0,.px-md-0{padding-right:0!important}.pb-md-0,.py-md-0{padding-bottom:0!important}.pl-md-0,.px-md-0{padding-left:0!important}.p-md-1{padding:8px!important}.pt-md-1,.py-md-1{padding-top:8px!important}.pr-md-1,.px-md-1{padding-right:8px!important}.pb-md-1,.py-md-1{padding-bottom:8px!important}.pl-md-1,.px-md-1{padding-left:8px!important}.p-md-2{padding:16px!important}.pt-md-2,.py-md-2{padding-top:16px!important}.pr-md-2,.px-md-2{padding-right:16px!important}.pb-md-2,.py-md-2{padding-bottom:16px!important}.pl-md-2,.px-md-2{padding-left:16px!important}.p-md-3{padding:24px!important}.pt-md-3,.py-md-3{padding-top:24px!important}.pr-md-3,.px-md-3{padding-right:24px!important}.pb-md-3,.py-md-3{padding-bottom:24px!important}.pl-md-3,.px-md-3{padding-left:24px!important}.p-md-4{padding:32px!important}.pt-md-4,.py-md-4{padding-top:32px!important}.pr-md-4,.px-md-4{padding-right:32px!important}.pb-md-4,.py-md-4{padding-bottom:32px!important}.pl-md-4,.px-md-4{padding-left:32px!important}.p-md-5{padding:40px!important}.pt-md-5,.py-md-5{padding-top:40px!important}.pr-md-5,.px-md-5{padding-right:40px!important}.pb-md-5,.py-md-5{padding-bottom:40px!important}.pl-md-5,.px-md-5{padding-left:40px!important}.p-md-6{padding:48px!important}.pt-md-6,.py-md-6{padding-top:48px!important}.pr-md-6,.px-md-6{padding-right:48px!important}.pb-md-6,.py-md-6{padding-bottom:48px!important}.pl-md-6,.px-md-6{padding-left:48px!important}.p-md-7{padding:56px!important}.pt-md-7,.py-md-7{padding-top:56px!important}.pr-md-7,.px-md-7{padding-right:56px!important}.pb-md-7,.py-md-7{padding-bottom:56px!important}.pl-md-7,.px-md-7{padding-left:56px!important}.p-md-8{padding:64px!important}.pt-md-8,.py-md-8{padding-top:64px!important}.pr-md-8,.px-md-8{padding-right:64px!important}.pb-md-8,.py-md-8{padding-bottom:64px!important}.pl-md-8,.px-md-8{padding-left:64px!important}.p-md-9{padding:72px!important}.pt-md-9,.py-md-9{padding-top:72px!important}.pr-md-9,.px-md-9{padding-right:72px!important}.pb-md-9,.py-md-9{padding-bottom:72px!important}.pl-md-9,.px-md-9{padding-left:72px!important}.p-md-10{padding:80px!important}.pt-md-10,.py-md-10{padding-top:80px!important}.pr-md-10,.px-md-10{padding-right:80px!important}.pb-md-10,.py-md-10{padding-bottom:80px!important}.pl-md-10,.px-md-10{padding-left:80px!important}.p-md-12{padding:96px!important}.pt-md-12,.py-md-12{padding-top:96px!important}.pr-md-12,.px-md-12{padding-right:96px!important}.pb-md-12,.py-md-12{padding-bottom:96px!important}.pl-md-12,.px-md-12{padding-left:96px!important}.p-md-15{padding:120px!important}.pt-md-15,.py-md-15{padding-top:120px!important}.pr-md-15,.px-md-15{padding-right:120px!important}.pb-md-15,.py-md-15{padding-bottom:120px!important}.pl-md-15,.px-md-15{padding-left:120px!important}.m-md-n1{margin:-8px!important}.mt-md-n1,.my-md-n1{margin-top:-8px!important}.mr-md-n1,.mx-md-n1{margin-right:-8px!important}.mb-md-n1,.my-md-n1{margin-bottom:-8px!important}.ml-md-n1,.mx-md-n1{margin-left:-8px!important}.m-md-n2{margin:-16px!important}.mt-md-n2,.my-md-n2{margin-top:-16px!important}.mr-md-n2,.mx-md-n2{margin-right:-16px!important}.mb-md-n2,.my-md-n2{margin-bottom:-16px!important}.ml-md-n2,.mx-md-n2{margin-left:-16px!important}.m-md-n3{margin:-24px!important}.mt-md-n3,.my-md-n3{margin-top:-24px!important}.mr-md-n3,.mx-md-n3{margin-right:-24px!important}.mb-md-n3,.my-md-n3{margin-bottom:-24px!important}.ml-md-n3,.mx-md-n3{margin-left:-24px!important}.m-md-n4{margin:-32px!important}.mt-md-n4,.my-md-n4{margin-top:-32px!important}.mr-md-n4,.mx-md-n4{margin-right:-32px!important}.mb-md-n4,.my-md-n4{margin-bottom:-32px!important}.ml-md-n4,.mx-md-n4{margin-left:-32px!important}.m-md-n5{margin:-40px!important}.mt-md-n5,.my-md-n5{margin-top:-40px!important}.mr-md-n5,.mx-md-n5{margin-right:-40px!important}.mb-md-n5,.my-md-n5{margin-bottom:-40px!important}.ml-md-n5,.mx-md-n5{margin-left:-40px!important}.m-md-n6{margin:-48px!important}.mt-md-n6,.my-md-n6{margin-top:-48px!important}.mr-md-n6,.mx-md-n6{margin-right:-48px!important}.mb-md-n6,.my-md-n6{margin-bottom:-48px!important}.ml-md-n6,.mx-md-n6{margin-left:-48px!important}.m-md-n7{margin:-56px!important}.mt-md-n7,.my-md-n7{margin-top:-56px!important}.mr-md-n7,.mx-md-n7{margin-right:-56px!important}.mb-md-n7,.my-md-n7{margin-bottom:-56px!important}.ml-md-n7,.mx-md-n7{margin-left:-56px!important}.m-md-n8{margin:-64px!important}.mt-md-n8,.my-md-n8{margin-top:-64px!important}.mr-md-n8,.mx-md-n8{margin-right:-64px!important}.mb-md-n8,.my-md-n8{margin-bottom:-64px!important}.ml-md-n8,.mx-md-n8{margin-left:-64px!important}.m-md-n9{margin:-72px!important}.mt-md-n9,.my-md-n9{margin-top:-72px!important}.mr-md-n9,.mx-md-n9{margin-right:-72px!important}.mb-md-n9,.my-md-n9{margin-bottom:-72px!important}.ml-md-n9,.mx-md-n9{margin-left:-72px!important}.m-md-n10{margin:-80px!important}.mt-md-n10,.my-md-n10{margin-top:-80px!important}.mr-md-n10,.mx-md-n10{margin-right:-80px!important}.mb-md-n10,.my-md-n10{margin-bottom:-80px!important}.ml-md-n10,.mx-md-n10{margin-left:-80px!important}.m-md-n12{margin:-96px!important}.mt-md-n12,.my-md-n12{margin-top:-96px!important}.mr-md-n12,.mx-md-n12{margin-right:-96px!important}.mb-md-n12,.my-md-n12{margin-bottom:-96px!important}.ml-md-n12,.mx-md-n12{margin-left:-96px!important}.m-md-n15{margin:-120px!important}.mt-md-n15,.my-md-n15{margin-top:-120px!important}.mr-md-n15,.mx-md-n15{margin-right:-120px!important}.mb-md-n15,.my-md-n15{margin-bottom:-120px!important}.ml-md-n15,.mx-md-n15{margin-left:-120px!important}.m-md-auto{margin:auto!important}.mt-md-auto,.my-md-auto{margin-top:auto!important}.mr-md-auto,.mx-md-auto{margin-right:auto!important}.mb-md-auto,.my-md-auto{margin-bottom:auto!important}.ml-md-auto,.mx-md-auto{margin-left:auto!important}}@media(min-width:980px){.m-lg-0{margin:0!important}.mt-lg-0,.my-lg-0{margin-top:0!important}.mr-lg-0,.mx-lg-0{margin-right:0!important}.mb-lg-0,.my-lg-0{margin-bottom:0!important}.ml-lg-0,.mx-lg-0{margin-left:0!important}.m-lg-1{margin:8px!important}.mt-lg-1,.my-lg-1{margin-top:8px!important}.mr-lg-1,.mx-lg-1{margin-right:8px!important}.mb-lg-1,.my-lg-1{margin-bottom:8px!important}.ml-lg-1,.mx-lg-1{margin-left:8px!important}.m-lg-2{margin:16px!important}.mt-lg-2,.my-lg-2{margin-top:16px!important}.mr-lg-2,.mx-lg-2{margin-right:16px!important}.mb-lg-2,.my-lg-2{margin-bottom:16px!important}.ml-lg-2,.mx-lg-2{margin-left:16px!important}.m-lg-3{margin:24px!important}.mt-lg-3,.my-lg-3{margin-top:24px!important}.mr-lg-3,.mx-lg-3{margin-right:24px!important}.mb-lg-3,.my-lg-3{margin-bottom:24px!important}.ml-lg-3,.mx-lg-3{margin-left:24px!important}.m-lg-4{margin:32px!important}.mt-lg-4,.my-lg-4{margin-top:32px!important}.mr-lg-4,.mx-lg-4{margin-right:32px!important}.mb-lg-4,.my-lg-4{margin-bottom:32px!important}.ml-lg-4,.mx-lg-4{margin-left:32px!important}.m-lg-5{margin:40px!important}.mt-lg-5,.my-lg-5{margin-top:40px!important}.mr-lg-5,.mx-lg-5{margin-right:40px!important}.mb-lg-5,.my-lg-5{margin-bottom:40px!important}.ml-lg-5,.mx-lg-5{margin-left:40px!important}.m-lg-6{margin:48px!important}.mt-lg-6,.my-lg-6{margin-top:48px!important}.mr-lg-6,.mx-lg-6{margin-right:48px!important}.mb-lg-6,.my-lg-6{margin-bottom:48px!important}.ml-lg-6,.mx-lg-6{margin-left:48px!important}.m-lg-7{margin:56px!important}.mt-lg-7,.my-lg-7{margin-top:56px!important}.mr-lg-7,.mx-lg-7{margin-right:56px!important}.mb-lg-7,.my-lg-7{margin-bottom:56px!important}.ml-lg-7,.mx-lg-7{margin-left:56px!important}.m-lg-8{margin:64px!important}.mt-lg-8,.my-lg-8{margin-top:64px!important}.mr-lg-8,.mx-lg-8{margin-right:64px!important}.mb-lg-8,.my-lg-8{margin-bottom:64px!important}.ml-lg-8,.mx-lg-8{margin-left:64px!important}.m-lg-9{margin:72px!important}.mt-lg-9,.my-lg-9{margin-top:72px!important}.mr-lg-9,.mx-lg-9{margin-right:72px!important}.mb-lg-9,.my-lg-9{margin-bottom:72px!important}.ml-lg-9,.mx-lg-9{margin-left:72px!important}.m-lg-10{margin:80px!important}.mt-lg-10,.my-lg-10{margin-top:80px!important}.mr-lg-10,.mx-lg-10{margin-right:80px!important}.mb-lg-10,.my-lg-10{margin-bottom:80px!important}.ml-lg-10,.mx-lg-10{margin-left:80px!important}.m-lg-12{margin:96px!important}.mt-lg-12,.my-lg-12{margin-top:96px!important}.mr-lg-12,.mx-lg-12{margin-right:96px!important}.mb-lg-12,.my-lg-12{margin-bottom:96px!important}.ml-lg-12,.mx-lg-12{margin-left:96px!important}.m-lg-15{margin:120px!important}.mt-lg-15,.my-lg-15{margin-top:120px!important}.mr-lg-15,.mx-lg-15{margin-right:120px!important}.mb-lg-15,.my-lg-15{margin-bottom:120px!important}.ml-lg-15,.mx-lg-15{margin-left:120px!important}.p-lg-0{padding:0!important}.pt-lg-0,.py-lg-0{padding-top:0!important}.pr-lg-0,.px-lg-0{padding-right:0!important}.pb-lg-0,.py-lg-0{padding-bottom:0!important}.pl-lg-0,.px-lg-0{padding-left:0!important}.p-lg-1{padding:8px!important}.pt-lg-1,.py-lg-1{padding-top:8px!important}.pr-lg-1,.px-lg-1{padding-right:8px!important}.pb-lg-1,.py-lg-1{padding-bottom:8px!important}.pl-lg-1,.px-lg-1{padding-left:8px!important}.p-lg-2{padding:16px!important}.pt-lg-2,.py-lg-2{padding-top:16px!important}.pr-lg-2,.px-lg-2{padding-right:16px!important}.pb-lg-2,.py-lg-2{padding-bottom:16px!important}.pl-lg-2,.px-lg-2{padding-left:16px!important}.p-lg-3{padding:24px!important}.pt-lg-3,.py-lg-3{padding-top:24px!important}.pr-lg-3,.px-lg-3{padding-right:24px!important}.pb-lg-3,.py-lg-3{padding-bottom:24px!important}.pl-lg-3,.px-lg-3{padding-left:24px!important}.p-lg-4{padding:32px!important}.pt-lg-4,.py-lg-4{padding-top:32px!important}.pr-lg-4,.px-lg-4{padding-right:32px!important}.pb-lg-4,.py-lg-4{padding-bottom:32px!important}.pl-lg-4,.px-lg-4{padding-left:32px!important}.p-lg-5{padding:40px!important}.pt-lg-5,.py-lg-5{padding-top:40px!important}.pr-lg-5,.px-lg-5{padding-right:40px!important}.pb-lg-5,.py-lg-5{padding-bottom:40px!important}.pl-lg-5,.px-lg-5{padding-left:40px!important}.p-lg-6{padding:48px!important}.pt-lg-6,.py-lg-6{padding-top:48px!important}.pr-lg-6,.px-lg-6{padding-right:48px!important}.pb-lg-6,.py-lg-6{padding-bottom:48px!important}.pl-lg-6,.px-lg-6{padding-left:48px!important}.p-lg-7{padding:56px!important}.pt-lg-7,.py-lg-7{padding-top:56px!important}.pr-lg-7,.px-lg-7{padding-right:56px!important}.pb-lg-7,.py-lg-7{padding-bottom:56px!important}.pl-lg-7,.px-lg-7{padding-left:56px!important}.p-lg-8{padding:64px!important}.pt-lg-8,.py-lg-8{padding-top:64px!important}.pr-lg-8,.px-lg-8{padding-right:64px!important}.pb-lg-8,.py-lg-8{padding-bottom:64px!important}.pl-lg-8,.px-lg-8{padding-left:64px!important}.p-lg-9{padding:72px!important}.pt-lg-9,.py-lg-9{padding-top:72px!important}.pr-lg-9,.px-lg-9{padding-right:72px!important}.pb-lg-9,.py-lg-9{padding-bottom:72px!important}.pl-lg-9,.px-lg-9{padding-left:72px!important}.p-lg-10{padding:80px!important}.pt-lg-10,.py-lg-10{padding-top:80px!important}.pr-lg-10,.px-lg-10{padding-right:80px!important}.pb-lg-10,.py-lg-10{padding-bottom:80px!important}.pl-lg-10,.px-lg-10{padding-left:80px!important}.p-lg-12{padding:96px!important}.pt-lg-12,.py-lg-12{padding-top:96px!important}.pr-lg-12,.px-lg-12{padding-right:96px!important}.pb-lg-12,.py-lg-12{padding-bottom:96px!important}.pl-lg-12,.px-lg-12{padding-left:96px!important}.p-lg-15{padding:120px!important}.pt-lg-15,.py-lg-15{padding-top:120px!important}.pr-lg-15,.px-lg-15{padding-right:120px!important}.pb-lg-15,.py-lg-15{padding-bottom:120px!important}.pl-lg-15,.px-lg-15{padding-left:120px!important}.m-lg-n1{margin:-8px!important}.mt-lg-n1,.my-lg-n1{margin-top:-8px!important}.mr-lg-n1,.mx-lg-n1{margin-right:-8px!important}.mb-lg-n1,.my-lg-n1{margin-bottom:-8px!important}.ml-lg-n1,.mx-lg-n1{margin-left:-8px!important}.m-lg-n2{margin:-16px!important}.mt-lg-n2,.my-lg-n2{margin-top:-16px!important}.mr-lg-n2,.mx-lg-n2{margin-right:-16px!important}.mb-lg-n2,.my-lg-n2{margin-bottom:-16px!important}.ml-lg-n2,.mx-lg-n2{margin-left:-16px!important}.m-lg-n3{margin:-24px!important}.mt-lg-n3,.my-lg-n3{margin-top:-24px!important}.mr-lg-n3,.mx-lg-n3{margin-right:-24px!important}.mb-lg-n3,.my-lg-n3{margin-bottom:-24px!important}.ml-lg-n3,.mx-lg-n3{margin-left:-24px!important}.m-lg-n4{margin:-32px!important}.mt-lg-n4,.my-lg-n4{margin-top:-32px!important}.mr-lg-n4,.mx-lg-n4{margin-right:-32px!important}.mb-lg-n4,.my-lg-n4{margin-bottom:-32px!important}.ml-lg-n4,.mx-lg-n4{margin-left:-32px!important}.m-lg-n5{margin:-40px!important}.mt-lg-n5,.my-lg-n5{margin-top:-40px!important}.mr-lg-n5,.mx-lg-n5{margin-right:-40px!important}.mb-lg-n5,.my-lg-n5{margin-bottom:-40px!important}.ml-lg-n5,.mx-lg-n5{margin-left:-40px!important}.m-lg-n6{margin:-48px!important}.mt-lg-n6,.my-lg-n6{margin-top:-48px!important}.mr-lg-n6,.mx-lg-n6{margin-right:-48px!important}.mb-lg-n6,.my-lg-n6{margin-bottom:-48px!important}.ml-lg-n6,.mx-lg-n6{margin-left:-48px!important}.m-lg-n7{margin:-56px!important}.mt-lg-n7,.my-lg-n7{margin-top:-56px!important}.mr-lg-n7,.mx-lg-n7{margin-right:-56px!important}.mb-lg-n7,.my-lg-n7{margin-bottom:-56px!important}.ml-lg-n7,.mx-lg-n7{margin-left:-56px!important}.m-lg-n8{margin:-64px!important}.mt-lg-n8,.my-lg-n8{margin-top:-64px!important}.mr-lg-n8,.mx-lg-n8{margin-right:-64px!important}.mb-lg-n8,.my-lg-n8{margin-bottom:-64px!important}.ml-lg-n8,.mx-lg-n8{margin-left:-64px!important}.m-lg-n9{margin:-72px!important}.mt-lg-n9,.my-lg-n9{margin-top:-72px!important}.mr-lg-n9,.mx-lg-n9{margin-right:-72px!important}.mb-lg-n9,.my-lg-n9{margin-bottom:-72px!important}.ml-lg-n9,.mx-lg-n9{margin-left:-72px!important}.m-lg-n10{margin:-80px!important}.mt-lg-n10,.my-lg-n10{margin-top:-80px!important}.mr-lg-n10,.mx-lg-n10{margin-right:-80px!important}.mb-lg-n10,.my-lg-n10{margin-bottom:-80px!important}.ml-lg-n10,.mx-lg-n10{margin-left:-80px!important}.m-lg-n12{margin:-96px!important}.mt-lg-n12,.my-lg-n12{margin-top:-96px!important}.mr-lg-n12,.mx-lg-n12{margin-right:-96px!important}.mb-lg-n12,.my-lg-n12{margin-bottom:-96px!important}.ml-lg-n12,.mx-lg-n12{margin-left:-96px!important}.m-lg-n15{margin:-120px!important}.mt-lg-n15,.my-lg-n15{margin-top:-120px!important}.mr-lg-n15,.mx-lg-n15{margin-right:-120px!important}.mb-lg-n15,.my-lg-n15{margin-bottom:-120px!important}.ml-lg-n15,.mx-lg-n15{margin-left:-120px!important}.m-lg-auto{margin:auto!important}.mt-lg-auto,.my-lg-auto{margin-top:auto!important}.mr-lg-auto,.mx-lg-auto{margin-right:auto!important}.mb-lg-auto,.my-lg-auto{margin-bottom:auto!important}.ml-lg-auto,.mx-lg-auto{margin-left:auto!important}}@media(min-width:1240px){.m-xl-0{margin:0!important}.mt-xl-0,.my-xl-0{margin-top:0!important}.mr-xl-0,.mx-xl-0{margin-right:0!important}.mb-xl-0,.my-xl-0{margin-bottom:0!important}.ml-xl-0,.mx-xl-0{margin-left:0!important}.m-xl-1{margin:8px!important}.mt-xl-1,.my-xl-1{margin-top:8px!important}.mr-xl-1,.mx-xl-1{margin-right:8px!important}.mb-xl-1,.my-xl-1{margin-bottom:8px!important}.ml-xl-1,.mx-xl-1{margin-left:8px!important}.m-xl-2{margin:16px!important}.mt-xl-2,.my-xl-2{margin-top:16px!important}.mr-xl-2,.mx-xl-2{margin-right:16px!important}.mb-xl-2,.my-xl-2{margin-bottom:16px!important}.ml-xl-2,.mx-xl-2{margin-left:16px!important}.m-xl-3{margin:24px!important}.mt-xl-3,.my-xl-3{margin-top:24px!important}.mr-xl-3,.mx-xl-3{margin-right:24px!important}.mb-xl-3,.my-xl-3{margin-bottom:24px!important}.ml-xl-3,.mx-xl-3{margin-left:24px!important}.m-xl-4{margin:32px!important}.mt-xl-4,.my-xl-4{margin-top:32px!important}.mr-xl-4,.mx-xl-4{margin-right:32px!important}.mb-xl-4,.my-xl-4{margin-bottom:32px!important}.ml-xl-4,.mx-xl-4{margin-left:32px!important}.m-xl-5{margin:40px!important}.mt-xl-5,.my-xl-5{margin-top:40px!important}.mr-xl-5,.mx-xl-5{margin-right:40px!important}.mb-xl-5,.my-xl-5{margin-bottom:40px!important}.ml-xl-5,.mx-xl-5{margin-left:40px!important}.m-xl-6{margin:48px!important}.mt-xl-6,.my-xl-6{margin-top:48px!important}.mr-xl-6,.mx-xl-6{margin-right:48px!important}.mb-xl-6,.my-xl-6{margin-bottom:48px!important}.ml-xl-6,.mx-xl-6{margin-left:48px!important}.m-xl-7{margin:56px!important}.mt-xl-7,.my-xl-7{margin-top:56px!important}.mr-xl-7,.mx-xl-7{margin-right:56px!important}.mb-xl-7,.my-xl-7{margin-bottom:56px!important}.ml-xl-7,.mx-xl-7{margin-left:56px!important}.m-xl-8{margin:64px!important}.mt-xl-8,.my-xl-8{margin-top:64px!important}.mr-xl-8,.mx-xl-8{margin-right:64px!important}.mb-xl-8,.my-xl-8{margin-bottom:64px!important}.ml-xl-8,.mx-xl-8{margin-left:64px!important}.m-xl-9{margin:72px!important}.mt-xl-9,.my-xl-9{margin-top:72px!important}.mr-xl-9,.mx-xl-9{margin-right:72px!important}.mb-xl-9,.my-xl-9{margin-bottom:72px!important}.ml-xl-9,.mx-xl-9{margin-left:72px!important}.m-xl-10{margin:80px!important}.mt-xl-10,.my-xl-10{margin-top:80px!important}.mr-xl-10,.mx-xl-10{margin-right:80px!important}.mb-xl-10,.my-xl-10{margin-bottom:80px!important}.ml-xl-10,.mx-xl-10{margin-left:80px!important}.m-xl-12{margin:96px!important}.mt-xl-12,.my-xl-12{margin-top:96px!important}.mr-xl-12,.mx-xl-12{margin-right:96px!important}.mb-xl-12,.my-xl-12{margin-bottom:96px!important}.ml-xl-12,.mx-xl-12{margin-left:96px!important}.m-xl-15{margin:120px!important}.mt-xl-15,.my-xl-15{margin-top:120px!important}.mr-xl-15,.mx-xl-15{margin-right:120px!important}.mb-xl-15,.my-xl-15{margin-bottom:120px!important}.ml-xl-15,.mx-xl-15{margin-left:120px!important}.p-xl-0{padding:0!important}.pt-xl-0,.py-xl-0{padding-top:0!important}.pr-xl-0,.px-xl-0{padding-right:0!important}.pb-xl-0,.py-xl-0{padding-bottom:0!important}.pl-xl-0,.px-xl-0{padding-left:0!important}.p-xl-1{padding:8px!important}.pt-xl-1,.py-xl-1{padding-top:8px!important}.pr-xl-1,.px-xl-1{padding-right:8px!important}.pb-xl-1,.py-xl-1{padding-bottom:8px!important}.pl-xl-1,.px-xl-1{padding-left:8px!important}.p-xl-2{padding:16px!important}.pt-xl-2,.py-xl-2{padding-top:16px!important}.pr-xl-2,.px-xl-2{padding-right:16px!important}.pb-xl-2,.py-xl-2{padding-bottom:16px!important}.pl-xl-2,.px-xl-2{padding-left:16px!important}.p-xl-3{padding:24px!important}.pt-xl-3,.py-xl-3{padding-top:24px!important}.pr-xl-3,.px-xl-3{padding-right:24px!important}.pb-xl-3,.py-xl-3{padding-bottom:24px!important}.pl-xl-3,.px-xl-3{padding-left:24px!important}.p-xl-4{padding:32px!important}.pt-xl-4,.py-xl-4{padding-top:32px!important}.pr-xl-4,.px-xl-4{padding-right:32px!important}.pb-xl-4,.py-xl-4{padding-bottom:32px!important}.pl-xl-4,.px-xl-4{padding-left:32px!important}.p-xl-5{padding:40px!important}.pt-xl-5,.py-xl-5{padding-top:40px!important}.pr-xl-5,.px-xl-5{padding-right:40px!important}.pb-xl-5,.py-xl-5{padding-bottom:40px!important}.pl-xl-5,.px-xl-5{padding-left:40px!important}.p-xl-6{padding:48px!important}.pt-xl-6,.py-xl-6{padding-top:48px!important}.pr-xl-6,.px-xl-6{padding-right:48px!important}.pb-xl-6,.py-xl-6{padding-bottom:48px!important}.pl-xl-6,.px-xl-6{padding-left:48px!important}.p-xl-7{padding:56px!important}.pt-xl-7,.py-xl-7{padding-top:56px!important}.pr-xl-7,.px-xl-7{padding-right:56px!important}.pb-xl-7,.py-xl-7{padding-bottom:56px!important}.pl-xl-7,.px-xl-7{padding-left:56px!important}.p-xl-8{padding:64px!important}.pt-xl-8,.py-xl-8{padding-top:64px!important}.pr-xl-8,.px-xl-8{padding-right:64px!important}.pb-xl-8,.py-xl-8{padding-bottom:64px!important}.pl-xl-8,.px-xl-8{padding-left:64px!important}.p-xl-9{padding:72px!important}.pt-xl-9,.py-xl-9{padding-top:72px!important}.pr-xl-9,.px-xl-9{padding-right:72px!important}.pb-xl-9,.py-xl-9{padding-bottom:72px!important}.pl-xl-9,.px-xl-9{padding-left:72px!important}.p-xl-10{padding:80px!important}.pt-xl-10,.py-xl-10{padding-top:80px!important}.pr-xl-10,.px-xl-10{padding-right:80px!important}.pb-xl-10,.py-xl-10{padding-bottom:80px!important}.pl-xl-10,.px-xl-10{padding-left:80px!important}.p-xl-12{padding:96px!important}.pt-xl-12,.py-xl-12{padding-top:96px!important}.pr-xl-12,.px-xl-12{padding-right:96px!important}.pb-xl-12,.py-xl-12{padding-bottom:96px!important}.pl-xl-12,.px-xl-12{padding-left:96px!important}.p-xl-15{padding:120px!important}.pt-xl-15,.py-xl-15{padding-top:120px!important}.pr-xl-15,.px-xl-15{padding-right:120px!important}.pb-xl-15,.py-xl-15{padding-bottom:120px!important}.pl-xl-15,.px-xl-15{padding-left:120px!important}.m-xl-n1{margin:-8px!important}.mt-xl-n1,.my-xl-n1{margin-top:-8px!important}.mr-xl-n1,.mx-xl-n1{margin-right:-8px!important}.mb-xl-n1,.my-xl-n1{margin-bottom:-8px!important}.ml-xl-n1,.mx-xl-n1{margin-left:-8px!important}.m-xl-n2{margin:-16px!important}.mt-xl-n2,.my-xl-n2{margin-top:-16px!important}.mr-xl-n2,.mx-xl-n2{margin-right:-16px!important}.mb-xl-n2,.my-xl-n2{margin-bottom:-16px!important}.ml-xl-n2,.mx-xl-n2{margin-left:-16px!important}.m-xl-n3{margin:-24px!important}.mt-xl-n3,.my-xl-n3{margin-top:-24px!important}.mr-xl-n3,.mx-xl-n3{margin-right:-24px!important}.mb-xl-n3,.my-xl-n3{margin-bottom:-24px!important}.ml-xl-n3,.mx-xl-n3{margin-left:-24px!important}.m-xl-n4{margin:-32px!important}.mt-xl-n4,.my-xl-n4{margin-top:-32px!important}.mr-xl-n4,.mx-xl-n4{margin-right:-32px!important}.mb-xl-n4,.my-xl-n4{margin-bottom:-32px!important}.ml-xl-n4,.mx-xl-n4{margin-left:-32px!important}.m-xl-n5{margin:-40px!important}.mt-xl-n5,.my-xl-n5{margin-top:-40px!important}.mr-xl-n5,.mx-xl-n5{margin-right:-40px!important}.mb-xl-n5,.my-xl-n5{margin-bottom:-40px!important}.ml-xl-n5,.mx-xl-n5{margin-left:-40px!important}.m-xl-n6{margin:-48px!important}.mt-xl-n6,.my-xl-n6{margin-top:-48px!important}.mr-xl-n6,.mx-xl-n6{margin-right:-48px!important}.mb-xl-n6,.my-xl-n6{margin-bottom:-48px!important}.ml-xl-n6,.mx-xl-n6{margin-left:-48px!important}.m-xl-n7{margin:-56px!important}.mt-xl-n7,.my-xl-n7{margin-top:-56px!important}.mr-xl-n7,.mx-xl-n7{margin-right:-56px!important}.mb-xl-n7,.my-xl-n7{margin-bottom:-56px!important}.ml-xl-n7,.mx-xl-n7{margin-left:-56px!important}.m-xl-n8{margin:-64px!important}.mt-xl-n8,.my-xl-n8{margin-top:-64px!important}.mr-xl-n8,.mx-xl-n8{margin-right:-64px!important}.mb-xl-n8,.my-xl-n8{margin-bottom:-64px!important}.ml-xl-n8,.mx-xl-n8{margin-left:-64px!important}.m-xl-n9{margin:-72px!important}.mt-xl-n9,.my-xl-n9{margin-top:-72px!important}.mr-xl-n9,.mx-xl-n9{margin-right:-72px!important}.mb-xl-n9,.my-xl-n9{margin-bottom:-72px!important}.ml-xl-n9,.mx-xl-n9{margin-left:-72px!important}.m-xl-n10{margin:-80px!important}.mt-xl-n10,.my-xl-n10{margin-top:-80px!important}.mr-xl-n10,.mx-xl-n10{margin-right:-80px!important}.mb-xl-n10,.my-xl-n10{margin-bottom:-80px!important}.ml-xl-n10,.mx-xl-n10{margin-left:-80px!important}.m-xl-n12{margin:-96px!important}.mt-xl-n12,.my-xl-n12{margin-top:-96px!important}.mr-xl-n12,.mx-xl-n12{margin-right:-96px!important}.mb-xl-n12,.my-xl-n12{margin-bottom:-96px!important}.ml-xl-n12,.mx-xl-n12{margin-left:-96px!important}.m-xl-n15{margin:-120px!important}.mt-xl-n15,.my-xl-n15{margin-top:-120px!important}.mr-xl-n15,.mx-xl-n15{margin-right:-120px!important}.mb-xl-n15,.my-xl-n15{margin-bottom:-120px!important}.ml-xl-n15,.mx-xl-n15{margin-left:-120px!important}.m-xl-auto{margin:auto!important}.mt-xl-auto,.my-xl-auto{margin-top:auto!important}.mr-xl-auto,.mx-xl-auto{margin-right:auto!important}.mb-xl-auto,.my-xl-auto{margin-bottom:auto!important}.ml-xl-auto,.mx-xl-auto{margin-left:auto!important}}.text-monospace{font-family:SFMono-Regular,Menlo,Monaco,Consolas,Liberation Mono,Courier New,monospace!important}.text-justify{text-align:justify!important}.text-wrap{white-space:normal!important}.text-nowrap{white-space:nowrap!important}.text-truncate{overflow:hidden;text-overflow:ellipsis;white-space:nowrap}.text-left{text-align:left!important}.text-right{text-align:right!important}.text-center{text-align:center!important}@media(min-width:400px){.text-xs-left{text-align:left!important}.text-xs-right{text-align:right!important}.text-xs-center{text-align:center!important}}@media(min-width:616px){.text-sm-left{text-align:left!important}.text-sm-right{text-align:right!important}.text-sm-center{text-align:center!important}}@media(min-width:768px){.text-md-left{text-align:left!important}.text-md-right{text-align:right!important}.text-md-center{text-align:center!important}}@media(min-width:980px){.text-lg-left{text-align:left!important}.text-lg-right{text-align:right!important}.text-lg-center{text-align:center!important}}@media(min-width:1240px){.text-xl-left{text-align:left!important}.text-xl-right{text-align:right!important}.text-xl-center{text-align:center!important}}.text-lowercase{text-transform:lowercase!important}.text-uppercase{text-transform:uppercase!important}.text-capitalize{text-transform:capitalize!important}.font-weight-light{font-weight:300!important}.font-weight-lighter{font-weight:lighter!important}.font-weight-normal{font-weight:400!important}.font-weight-bold{font-weight:700!important}.font-weight-bolder{font-weight:bolder!important}.font-italic{font-style:italic!important}.text-primary{color:#fc0!important}a.text-primary:focus,a.text-primary:hover{color:#b38f00!important}.text-secondary{color:#212529!important}a.text-secondary:focus,a.text-secondary:hover{color:#000!important}.text-success{color:#28a745!important}a.text-success:focus,a.text-success:hover{color:#19692c!important}.text-info{color:#17a2b8!important}a.text-info:focus,a.text-info:hover{color:#0f6674!important}.text-warning{color:#ffc107!important}a.text-warning:focus,a.text-warning:hover{color:#ba8b00!important}.text-danger{color:#dc3545!important}a.text-danger:focus,a.text-danger:hover{color:#a71d2a!important}.text-light{color:#f1f6f9!important}a.text-light:focus,a.text-light:hover{color:#bbd4e2!important}.text-dark{color:#495057!important}a.text-dark:focus,a.text-dark:hover{color:#262a2d!important}.text-primary-light{color:#fffaf0!important}a.text-primary-light:focus,a.text-primary-light:hover{color:#ffe1a4!important}.text-secondary-light{color:#fff!important}a.text-secondary-light:focus,a.text-secondary-light:hover{color:#d9d9d9!important}.text-tertiary{color:#257af4!important}a.text-tertiary:focus,a.text-tertiary:hover{color:#0a56c3!important}.text-tertiary-light{color:#e3f1fe!important}a.text-tertiary-light:focus,a.text-tertiary-light:hover{color:#99ccfb!important}.text-white{color:#fff!important}a.text-white:focus,a.text-white:hover{color:#d9d9d9!important}.text-black{color:#212529!important}a.text-black:focus,a.text-black:hover{color:#000!important}.text-blue{color:#257af4!important}a.text-blue:focus,a.text-blue:hover{color:#0a56c3!important}.text-light-blue{color:#e3f1fe!important}a.text-light-blue:focus,a.text-light-blue:hover{color:#99ccfb!important}.text-yellow{color:#fc0!important}a.text-yellow:focus,a.text-yellow:hover{color:#b38f00!important}.text-light-yellow{color:#fffaf0!important}a.text-light-yellow:focus,a.text-light-yellow:hover{color:#ffe1a4!important}.text-orange{color:#ff8c00!important}a.text-orange:focus,a.text-orange:hover{color:#b36200!important}.text-light-orange{color:#ffe4b5!important}a.text-light-orange:focus,a.text-light-orange:hover{color:#ffc869!important}.text-red{color:#ff3939!important}a.text-red:focus,a.text-red:hover{color:#ec0000!important}.text-light-red{color:#ffe4e1!important}a.text-light-red:focus,a.text-light-red:hover{color:#ff9f95!important}.text-medium{color:#d6dbdf!important}a.text-medium:focus,a.text-medium:hover{color:#abb5bd!important}.text-body{color:#212529!important}.text-muted{color:#6c757d!important}.text-black-50{color:rgba(33,37,41,.5)!important}.text-white-50{color:hsla(0,0%,100%,.5)!important}.text-hide{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.text-decoration-none{text-decoration:none!important}.text-break{word-break:break-word!important;overflow-wrap:break-word!important}.text-reset{color:inherit!important}.visible{visibility:visible!important}.invisible{visibility:hidden!important}@media print{*,:after,:before{text-shadow:none!important;box-shadow:none!important}a:not(.btn){text-decoration:underline}abbr[title]:after{content:" (" attr(title) ")"}pre{white-space:pre-wrap!important}blockquote,pre{border:1px solid #d6dbdf;page-break-inside:avoid}thead{display:table-header-group}img,tr{page-break-inside:avoid}h2,h3,p{orphans:3;widows:3}h2,h3{page-break-after:avoid}@page{size:a3}.container,body{min-width:980px!important}.navbar{display:none}.badge{border:1px solid #212529}.table{border-collapse:collapse!important}.table td,.table th{background-color:#fff!important}.table-bordered td,.table-bordered th{border:1px solid #dee2e6!important}.table-dark{color:inherit}.table-dark tbody+tbody,.table-dark td,.table-dark th,.table-dark thead th{border-color:#d6dbdf}.table .thead-dark th{color:inherit;border-color:#d6dbdf}} \ No newline at end of file diff --git a/website/css/greenhouse.css b/website/css/greenhouse.css new file mode 100644 index 00000000000..76812a169e8 --- /dev/null +++ b/website/css/greenhouse.css @@ -0,0 +1 @@ +#main{padding-bottom:0;padding-top:0}#wrapper{max-width:1078px;padding:0}body>#wrapper>#main>#wrapper>#content,body>#wrapper>#main>#wrapper>#logo,body>#wrapper>#main>#wrapper>h1{display:none}body>#wrapper>#main>#wrapper>#board_title{margin-top:0}body>#wrapper>#main>#logo{margin-top:80px}body>#wrapper>#main>:last-child{margin-bottom:120px} \ No newline at end of file diff --git a/website/css/main.css b/website/css/main.css index 9b676804eba..229b74cb3f7 100644 --- a/website/css/main.css +++ b/website/css/main.css @@ -1,1039 +1 @@ -@media screen and (max-width: 978.98px) { - .btn { - padding: 8px 16px; - } -} - -@media screen and (max-width: 978.98px) { - .btn-lg { - padding: 12px 24px; - } -} - -.btn-primary { - color: #212529; -} -.btn-primary:active, .btn-primary:hover { - color: #212529; -} - -.btn-outline-primary { - background: #fffaf0; - border-color: #ffcc00; - color: #212529; -} -.btn-outline-primary:active, .btn-outline-primary:hover { - background: #ffcc00; - border-color: #ffcc00; - color: #212529; -} - -.btn-secondary { - border-color: #212529; - color: #fff; -} -.btn-secondary:active, .btn-secondary:hover { - background: #fff; - border-color: #212529; - color: #212529; -} - -.btn-outline-secondary { - background: #fff; - border-color: #212529; - color: #212529; -} -.btn-outline-secondary:active, .btn-outline-secondary:hover { - background: #212529; - border-color: #212529; - color: #fff; -} - -.btn-tertiary { - border-color: #257af4; - color: #fff; -} -.btn-tertiary:active, .btn-tertiary:hover { - background: #257af4; - border-color: #257af4; - color: #fff; -} - -.btn-outline-tertiary { - background: #e3f1fe; - color: #257af4; -} -.btn-outline-tertiary:active, .btn-outline-tertiary:hover { - background: #257af4; - color: #fff; -} - -.btns { - align-items: center; - display: flex; - justify-content: center; -} -.btns .btn + .btn { - margin-left: 24px; -} -.btns .btn-lg + .btn-lg { - margin-left: 40px; -} - -.card { - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2); - overflow: hidden; - position: relative; - transition: box-shadow 0.2s, transform 0.2s; - width: 100%; -} -.card-body { - position: relative; - z-index: 10; -} - -.card.is-large .card-body { - padding: 40px; -} - -.card.bg-primary-light { - border-color: #ffcc00; -} - -.card.has-dark-border { - border-color: #6c757d; -} - -.card.has-pattern:before, -.card.has-pattern:after { - background-repeat: no-repeat; - background-size: auto 100%; - bottom: 0; - content: ""; - display: block; - position: absolute; - top: 0; - width: 72px; -} - -.card.has-pattern:before { - background-image: url(../images/backgrounds/bg-card-pattern-blue-1.png); - background-position: left top; - left: 0; -} - -.card.has-pattern:after { - background-image: url(../images/backgrounds/bg-card-pattern-blue-2.png); - background-position: right top; - right: 0; -} - -a.card:active, -a.card:hover, -.card.has-hover:active, -.card.has-hover:hover { - box-shadow: 0 12px 32px rgba(108, 117, 125, 0.2); - transform: translateY(-8px); -} - -a.card:after, -.card.has-hover:after, -.card.has-highlight:after { - content: ""; - display: block; - height: 8px; - margin-top: auto; - transition: background 0.2s; - width: 100%; -} - -a.card:active:after, -a.card:hover:after, -.card.has-hover:active:after, -.card.has-hover:hover:after, -.card.has-highlight:after { - background: #e3f1fe; -} - -.case-study-cards { - -moz-column-gap: 40px; - column-gap: 40px; - display: grid; - grid-template-columns: 1fr; - row-gap: 40px; - padding-bottom: 40px; - position: relative; -} -.case-study-cards > div { - align-items: stretch; - display: flex; -} -.case-study-cards:before { - background: #d6dbdf; - bottom: 0; - content: ""; - display: block; - left: 20px; - position: absolute; - top: 40px; - width: 100vw; -} -@media screen and (min-width: 980px) { - .case-study-cards { - grid-template-columns: repeat(2, minmax(0, 1fr)); - row-gap: 80px; - padding-bottom: 120px; - } - .case-study-cards:before { - left: -40px; - top: 120px; - } -} - -.case-study-card { - align-items: stretch; - flex-direction: row; - flex-shrink: 0; - left: 0; - transition: 0.2s box-shadow, 0.4s left, 0.4s width, 0s z-index; - transition-delay: 0s, 0.6s, 0.6s, 0s; - width: 100%; - z-index: 2; -} -@media screen and (max-width: 979.98px) { - .case-study-card .row { - min-height: 0 !important; - } -} -@media screen and (min-width: 980px) { - .case-study-card:active, .case-study-card:hover { - box-shadow: 0 12px 32px rgba(108, 117, 125, 0.2); - } - .case-study-card:not(.is-open) { - cursor: pointer; - } - .case-study-card.is-open { - transform: none !important; - transition-delay: 0s, 0s, 0s, 0s; - width: calc(200% + 40px); - z-index: 10; - } - .case-study-card.is-closing { - z-index: 10; - } - .case-study-card.open-left.is-open { - left: calc(-100% - 40px); - } - .case-study-card:before { - background: no-repeat url(../images/backgrounds/bg-card-pattern-red.png); - background-position: right center; - background-size: contain; - content: ""; - display: block; - height: calc(100% - 80px); - max-height: 224px; - max-width: 234px; - position: absolute; - right: 0; - top: 40px; - transform: translateX(30%); - transition: 0.4s transform; - transition-delay: 0.6s; - width: 100%; - z-index: 1; - } -} -@media screen and (min-width: 980px) and (min-width: 1240px) { - .case-study-card:before { - transform: translateX(10%); - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open:before { - transform: translateX(60%); - transition-delay: 0s; - } -} -@media screen and (min-width: 980px) { - .case-study-card-wrap { - align-items: stretch; - display: flex; - flex-shrink: 0; - min-height: 304px; - position: relative; - transition: 0.4s width; - transition-delay: 0.6s; - width: calc(200% + 42px); - z-index: 2; - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open .case-study-card-wrap { - transition-delay: 0s; - width: 100%; - } -} -@media screen and (min-width: 980px) { - .case-study-card-body { - display: flex; - flex-direction: column; - padding-right: 80px !important; - } - .case-study-card-body > .row { - align-self: stretch; - } -} -@media screen and (min-width: 980px) { - .case-study-card-toggle { - background: #fff; - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2); - border-radius: 100%; - cursor: pointer; - height: 56px; - position: relative; - width: 56px; - } - .case-study-card-toggle:before, .case-study-card-toggle:after { - background: #257af4; - content: ""; - display: block; - height: 4px; - left: calc(50% - 15px); - position: absolute; - top: calc(50% - 2px); - transition: opacity 0.2s, transform 0.2s; - width: 30px; - } - .case-study-card-toggle:after { - transform: rotate(90deg); - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open .case-study-card-toggle:before { - opacity: 0; - transform: rotate(-90deg); - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open .case-study-card-toggle:after { - transform: rotate(0); - } -} -@media screen and (min-width: 980px) { - .case-study-card .col-lg-3, -.case-study-card .col-lg-auto { - opacity: 0; - transform: translateX(24px); - transition: 0.4s opacity, 0.4s transform; - } -} -@media screen and (min-width: 980px) { - .case-study-card .col-lg-3 { - transition-delay: 0s; - } -} -@media screen and (min-width: 980px) { - .case-study-card .col-lg-auto { - transition-delay: 0.2s; - } -} -@media screen and (min-width: 980px) and (min-width: 980px) { - .case-study-card .col-lg-auto { - max-width: 605px; - width: calc(100% - 319px); - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open .col-lg-3, .case-study-card.is-open .col-lg-auto { - opacity: 1; - transform: none; - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open .col-lg-3 { - transition-delay: 0.4s; - } -} -@media screen and (min-width: 980px) { - .case-study-card.is-open .col-lg-auto { - transition-delay: 0.2s; - } -} - -.footer-copy { - white-space: nowrap; -} - -form .form-control { - border: 1px solid #6c757d; - border-radius: 6px; - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2); - color: #212529; - height: auto; - line-height: 20px; - min-height: 44px; - padding: 12px 16px; - width: 100%; -} -form .form-control:focus { - border-color: #212529; - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2); - color: #212529; -} -form .form-control::-moz-placeholder { - color: #6c757d; -} -form .form-control:-ms-input-placeholder { - color: #6c757d; -} -form .form-control::placeholder { - color: #6c757d; -} -form select.form-control { - -webkit-appearance: none; - -moz-appearance: none; - appearance: none; -} -form select.form-control:not([data-chosen]) { - color: #6c757d; -} -form .btn-secondary:active, -form .btn-secondary:hover { - color: #212529; - background: #ffcc00; - border-color: #ffcc00; -} - -.hero { - background-repeat: no-repeat; - background-position: center; - background-size: cover; - overflow: visible; - position: relative; -} -.hero-bg { - background-repeat: no-repeat; - background-position: center; - background-size: cover; - display: block; - height: 100%; - left: 50%; - position: absolute; - top: 0; - transform: translateX(-50%); - z-index: 1; -} -.hero > .container { - position: relative; - z-index: 2; -} - -.hero.has-offset { - margin-bottom: -160px; - padding-bottom: 160px; -} - -hr { - background: #f1f6f9; - border: 0; - display: block; - height: 4px; - margin: 0; - width: 100%; -} - -hr.is-small { - height: 2px; -} - -hr.is-large { - height: 8px; -} - -hr.is-medium { - background: #d6dbdf; -} - -hr.is-dark { - background: #495057; -} - -hr.is-yellow { - background: linear-gradient(90deg, #ff8c00 0, #ff8c00 8px, #ffcc00 16px, rgba(255, 204, 0, 0) 100%); - -webkit-clip-path: polygon(8px 100%, 0 100%, 0 0, 8px 0, 8px 100%, 16px 100%, 16px 0, 100% 0, 100% 100%); - clip-path: polygon(8px 100%, 0 100%, 0 0, 8px 0, 8px 100%, 16px 100%, 16px 0, 100% 0, 100% 100%); - height: 8px; -} - -.icon { - display: block; - height: 48px; - margin-bottom: 24px; - -o-object-fit: contain; - object-fit: contain; - -o-object-position: center; - object-position: center; -} -@media screen and (min-width: 576px) { - .icon { - height: 64px; - } -} -@media screen and (min-width: 980px) { - .icon { - height: 80px; - } -} - -img { - max-width: 100%; -} - -.kicker { - color: #6c757d; - font-family: "Hind Siliguri", sans-serif; - font-size: 0.875rem; - font-weight: 600; - letter-spacing: 1px; - margin: 0; -} - -@media screen and (max-width: 978.98px) { - .lead { - font-size: 1.125rem; - } -} - -.navbar-clickhouse { - border-bottom: 4px solid #f1f6f9 !important; - height: 142px !important; -} -.navbar-clickhouse > .container { - flex-wrap: wrap !important; -} -.navbar-super { - flex-shrink: 0; - width: 100%; -} -.navbar-super ul { - list-style: none; -} -.navbar-super li:not(:last-child) { - margin-bottom: 0; - margin-right: 24px; -} -.navbar-super a { - align-items: center; - color: #212529; - display: flex; - font-size: 0.875rem; -} -.navbar-super a:active, .navbar-super a:hover { - color: #257af4; - text-decoration: none; -} -.navbar-super img { - flex-shrink: 0; - margin-right: 4px; -} -.navbar-brand-clickhouse { - background: no-repeat url(/images/logo-clickhouse.svg); - background-size: contain; - flex-shrink: 0; - height: 28px; - margin-right: 48px; - padding: 0; - width: 180px; -} -.navbar-nav { - align-items: center; - height: 46px; -} -.navbar .nav-item:not(:last-child) { - margin-bottom: 0; - margin-right: 24px; -} -.navbar .nav-link { - color: #212529; -} -.navbar .nav-link:active, .navbar .nav-link:hover { - color: #257af4; -} -.navbar .navbar-nav { - flex-direction: row; -} -@media screen and (max-width: 978.98px) { - .navbar > .container { - padding-left: 20px; - padding-right: 20px; - } - .navbar .navbar-toggler { - height: 24px; - padding: 0; - width: 24px; - } - .navbar .navbar-toggler:focus { - outline: none; - } - .navbar .navbar-toggler-icon { - background: no-repeat url(../images/icons/icon-menu.svg); - background-position: center; - background-size: contain; - height: 24px; - width: 24px; - } - .navbar .navbar-collapse { - background: #fff; - border-bottom: 4px solid #f1f6f9; - height: 56px; - left: 0; - padding: 0 20px 16px; - position: absolute; - right: 0; - top: 100%; - } - .navbar .nav-link { - font-size: 0.875rem; - white-space: nowrap; - } -} -@media screen and (max-width: 615.98px) { - .navbar .navbar-collapse { - height: auto; - } - .navbar .navbar-nav { - flex-direction: column; - height: auto; - } - .navbar .nav-item:not(:last-child) { - margin-bottom: 16px; - margin-right: 0; - } -} -@media screen and (max-width: 399.98px) { - .navbar { - height: 80px; - } -} - -.page { - overflow: hidden; - width: 100vw; -} - -.photo-frame { - background: rgba(255, 255, 255, 0.6); - border-radius: 100%; - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2); - display: block; - margin-bottom: 24px; - max-width: 160px; - overflow: hidden; - position: relative; - width: 100%; -} -.photo-frame:before { - content: ""; - display: block; - padding-bottom: 100%; - width: 100%; -} -.photo-frame img { - display: block; - height: 100%; - left: 0; - -o-object-fit: contain; - object-fit: contain; - -o-object-position: center; - object-position: center; - position: absolute; - top: 0; - width: 100%; -} - -.pullquote { - position: relative; - width: 70%; -} -.pullquote:before { - background: no-repeat url(../images/backgrounds/bg-quotes.svg); - background-position: center; - background-size: 100%; - bottom: 0; - content: ""; - display: block; - mix-blend-mode: multiply; - position: absolute; - right: 56px; - top: 0; - width: calc(100% - 16px); - z-index: 2; -} -.pullquote-bg { - bottom: 0; - display: block; - position: absolute; - right: 0; - top: 0; - width: calc(50vw + 28.5714285714%); - z-index: 1; -} -.pullquote-body { - padding: 64px 40px 64px 0; - position: relative; - z-index: 3; -} -.pullquote-quote { - font-family: "Hind Siliguri", sans-serif; - font-size: 32px; - font-weight: 700; -} -.pullquote-citation { - font-size: 1.125rem; -} - -.section { - background-repeat: no-repeat; - background-position: center; - background-size: cover; - overflow: visible; - position: relative; -} -.section-bg { - background-repeat: no-repeat; - background-position: center; - background-size: cover; - display: block; - height: 100%; - left: 50%; - position: absolute; - top: 0; - transform: translateX(-50%); - z-index: 1; -} -.section > .container { - position: relative; - z-index: 2; -} - -.social-icons { - align-items: center; - display: flex; -} -.social-icons > a { - aspect-ratio: 24/24; - background: #6c757d; - display: block; - height: 24px; - width: 24px; - -webkit-mask-position: center; - mask-position: center; - -webkit-mask-repeat: no-repeat; - mask-repeat: no-repeat; - -webkit-mask-size: contain; - mask-size: contain; - transition: 0.2s background; -} -.social-icons > a:active, .social-icons > a:hover { - background: #212529; -} -.social-icons > a + a { - margin-left: 32px; -} -.social-icons-facebook { - -webkit-mask-image: url("/images/icons/icon-facebook-gray.svg"); - mask-image: url("/images/icons/icon-facebook-gray.svg"); -} -.social-icons-twitter { - -webkit-mask-image: url("/images/icons/icon-twitter-gray.svg"); - mask-image: url("/images/icons/icon-twitter-gray.svg"); - width: 31px; -} -.social-icons-linkedin { - -webkit-mask-image: url("/images/icons/icon-linkedin-gray.svg"); - mask-image: url("/images/icons/icon-linkedin-gray.svg"); -} -.social-icons-linkedin-alt { - -webkit-mask-image: url("/images/icons/icon-linkedin-alt-gray.svg"); - mask-image: url("/images/icons/icon-linkedin-alt-gray.svg"); -} - -.social-icons.size-small > a { - height: 20px; - width: 20px; -} -.social-icons.size-small > a:active, .social-icons.size-small > a:hover { - background: #212529; -} -.social-icons.size-small > a + a { - margin-left: 16px; -} - -.tabs { - position: relative; -} -.tabs:before { - background: #fff; - border-radius: 7px 7px 0 0; - content: ""; - display: block; - height: 8px; - left: 1px; - position: absolute; - right: 1px; - top: 68px; - z-index: 10; -} -@media screen and (min-width: 1240px) { - .tabs:before { - top: 76px; - } -} -.tabs-body { - background: #fff; - border-color: #6c757d; - border-radius: 8px; - border-style: solid; - border-width: 1px; - box-shadow: 0 8px 20px rgba(108, 117, 125, 0.2); - padding: 24px; -} -@media screen and (min-width: 980px) { - .tabs-body { - padding: 32px; - } -} -@media screen and (min-width: 1240px) { - .tabs-body { - padding: 40px; - } -} -.tabs .nav-tabs { - border-bottom: 0; - flex-wrap: nowrap; - height: 76px; - margin: -20px -20px -9px; - -webkit-mask-image: linear-gradient(90deg, transparent 0%, #000 20px, #000 calc(100% - 20px), transparent 100%); - mask-image: linear-gradient(90deg, transparent 0%, #000 20px, #000 calc(100% - 20px), transparent 100%); - overflow: scroll; - overflow-x: scroll; - overflow-y: visible; - padding: 20px 20px 0; - position: relative; -} -@media screen and (min-width: 940px) { - .tabs .nav-tabs { - overflow: visible; - } -} -@media screen and (min-width: 1240px) { - .tabs .nav-tabs { - height: 84px; - } -} -.tabs .nav-link { - align-items: center; - border-bottom: 0; - color: #6c757d; - display: flex; - font-size: 0.875rem; - flex-shrink: 0; - height: 56px; - justify-content: center; - padding: 0 12px 8px; - text-align: center; - white-space: nowrap; -} -@media screen and (min-width: 1240px) { - .tabs .nav-link { - height: 64px; - padding: 0 16px 8px; - } -} -.tabs .nav-link.active { - background: #fff; - box-shadow: 0 -4px 8px rgba(108, 117, 125, 0.1); - font-weight: 700; - padding: 0 16px 8px; -} -@media screen and (min-width: 980px) { - .tabs .nav-link.active { - padding: 0 24px 8px; - } -} -@media screen and (min-width: 1240px) { - .tabs .nav-link.active { - padding: 0 32px 8px; - } -} - -.tab-pane pre { - background: #212529; - border-radius: 16px; - color: #fff; - padding: 24px 16px; -} -@media screen and (min-width: 1240px) { - .tab-pane pre { - padding: 32px 24px; - } -} - -.trailing-link { - align-items: center; - color: #212529; - display: flex; - font-size: 0.875rem; - font-weight: 700; -} -.trailing-link:after { - background: no-repeat url(../images/icons/icon-arrow.svg); - background-position: right center; - background-size: contain; - content: ""; - display: block; - height: 12px; - transition: 0.2s transform; - width: 20px; -} -.trailing-link:active, .trailing-link:hover { - color: #212529; - text-decoration: none; -} -.trailing-link:active:after, .trailing-link:hover:after { - transform: translateX(8px); -} -.trailing-link.span-full:after { - margin-left: auto; -} - -ul { - list-style-type: square; - padding-left: 1.25em; -} -ul li:not(:last-child) { - margin-bottom: 16px; -} -ul li::marker { - color: #ff3939; -} - -ul.has-separators { - list-style: none; - padding: 0; -} -ul.has-separators li:not(:last-child) { - border-bottom: 4px solid #f1f6f9; - margin-bottom: 24px; - padding-bottom: 24px; -} - -.bg-gradient-secondary { - background-image: linear-gradient(58deg, #FF6443 3%, #FE561D 24%, #E32F0D 93%); -} - -.bg-gradient-light-orange { - background-image: linear-gradient(90deg, rgba(255, 203, 128, 0) 0%, #FFCB80 100%); -} - -.bg-offset-right { - bottom: 0; - left: -24px; - position: absolute; - top: 0; - width: calc(100vw + 24px); - z-index: -1; -} -@media screen and (min-width: 1240px) { - .bg-offset-right { - left: -96px; - width: calc(100vw + 96px); - } -} - -.bg-inset-right { - bottom: 0; - left: 40px; - position: absolute; - top: 0; - width: calc(100vw - 40px); - z-index: -1; -} -@media screen and (min-width: 980px) { - .bg-inset-right { - left: 96px; - width: calc(100vw - 96px); - } -} - -.has-border-left { - border-left: 8px solid #f1f6f9; - padding-left: 16px; -} - -.font-xl { - font-size: 1.25rem; -} - -.font-lg { - font-size: 1.125rem; -} - -.font-sm { - font-size: 0.875rem; -} - -.font-xs { - font-size: 0.625rem; -} - -.font-weight-semibold { - font-weight: 600; -} - -.display-5 { - color: #212529; - font-size: 20px; - font-weight: 500; -} - -.display-6 { - color: #212529; - font-size: 14px; - font-weight: 700; -} - -.text-decoration-underline { - text-decoration: underline; -} - -.text-upper { - text-transform: uppercase; -} -.base-hero { - height:22.5vw; - max-height:324px; - min-height:280px; -} -.index-hero { - background-image:url('/images/backgrounds/bg-hero-home.svg'); - height:68vw; - max-height:980px; - max-width:2448px; - width:170vw; -} -.other-hero { - background-image: url('/images/backgrounds/bg-hero.svg'); - max-width: 2448px; - width: 170vw; -} -.bg-footer-cta { - background-image:url('/images/backgrounds/bg-footer-cta.svg'); - width:2448px; -} -.quickstart-bg { - background-image:url('/images/backgrounds/bg-quick-start.svg'); - height:40vw; - top:220px; - width:170vw; -} +@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:flex;justify-content:center}.btns .btn+.btn{margin-left:24px}.btns .btn-lg+.btn-lg{margin-left:40px}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(10%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(60%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3,.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s}}@media screen and (min-width:980px){.case-study-card .col-lg-3{transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3,.case-study-card.is-open .col-lg-auto{opacity:1;transform:none}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{transition-delay:.4s}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page{overflow:hidden;width:100vw}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;overflow:hidden;position:relative;width:100%}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{color:#495057;list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file diff --git a/website/js/main.js b/website/js/main.js index 9466b6849ba..97bb490d9f1 100644 --- a/website/js/main.js +++ b/website/js/main.js @@ -1,158 +1 @@ -/******/ (function(modules) { // webpackBootstrap -/******/ // The module cache -/******/ var installedModules = {}; -/******/ -/******/ // The require function -/******/ function __webpack_require__(moduleId) { -/******/ -/******/ // Check if module is in cache -/******/ if(installedModules[moduleId]) { -/******/ return installedModules[moduleId].exports; -/******/ } -/******/ // Create a new module (and put it into the cache) -/******/ var module = installedModules[moduleId] = { -/******/ i: moduleId, -/******/ l: false, -/******/ exports: {} -/******/ }; -/******/ -/******/ // Execute the module function -/******/ modules[moduleId].call(module.exports, module, module.exports, __webpack_require__); -/******/ -/******/ // Flag the module as loaded -/******/ module.l = true; -/******/ -/******/ // Return the exports of the module -/******/ return module.exports; -/******/ } -/******/ -/******/ -/******/ // expose the modules object (__webpack_modules__) -/******/ __webpack_require__.m = modules; -/******/ -/******/ // expose the module cache -/******/ __webpack_require__.c = installedModules; -/******/ -/******/ // define getter function for harmony exports -/******/ __webpack_require__.d = function(exports, name, getter) { -/******/ if(!__webpack_require__.o(exports, name)) { -/******/ Object.defineProperty(exports, name, { enumerable: true, get: getter }); -/******/ } -/******/ }; -/******/ -/******/ // define __esModule on exports -/******/ __webpack_require__.r = function(exports) { -/******/ if(typeof Symbol !== 'undefined' && Symbol.toStringTag) { -/******/ Object.defineProperty(exports, Symbol.toStringTag, { value: 'Module' }); -/******/ } -/******/ Object.defineProperty(exports, '__esModule', { value: true }); -/******/ }; -/******/ -/******/ // create a fake namespace object -/******/ // mode & 1: value is a module id, require it -/******/ // mode & 2: merge all properties of value into the ns -/******/ // mode & 4: return value when already ns object -/******/ // mode & 8|1: behave like require -/******/ __webpack_require__.t = function(value, mode) { -/******/ if(mode & 1) value = __webpack_require__(value); -/******/ if(mode & 8) return value; -/******/ if((mode & 4) && typeof value === 'object' && value && value.__esModule) return value; -/******/ var ns = Object.create(null); -/******/ __webpack_require__.r(ns); -/******/ Object.defineProperty(ns, 'default', { enumerable: true, value: value }); -/******/ if(mode & 2 && typeof value != 'string') for(var key in value) __webpack_require__.d(ns, key, function(key) { return value[key]; }.bind(null, key)); -/******/ return ns; -/******/ }; -/******/ -/******/ // getDefaultExport function for compatibility with non-harmony modules -/******/ __webpack_require__.n = function(module) { -/******/ var getter = module && module.__esModule ? -/******/ function getDefault() { return module['default']; } : -/******/ function getModuleExports() { return module; }; -/******/ __webpack_require__.d(getter, 'a', getter); -/******/ return getter; -/******/ }; -/******/ -/******/ // Object.prototype.hasOwnProperty.call -/******/ __webpack_require__.o = function(object, property) { return Object.prototype.hasOwnProperty.call(object, property); }; -/******/ -/******/ // __webpack_public_path__ -/******/ __webpack_require__.p = ""; -/******/ -/******/ -/******/ // Load entry module and return exports -/******/ return __webpack_require__(__webpack_require__.s = 0); -/******/ }) -/************************************************************************/ -/******/ ({ - -/***/ "../../website/src/js/components/case-study-card.js": -/*!**************************************************************************************!*\ - !*** /Users/cody/Sites/tech.clickhouse/website/src/js/components/case-study-card.js ***! - \**************************************************************************************/ -/*! no static exports found */ -/***/ (function(module, exports) { - -eval("function _classCallCheck(instance, Constructor) { if (!(instance instanceof Constructor)) { throw new TypeError(\"Cannot call a class as a function\"); } }\n\nfunction _defineProperties(target, props) { for (var i = 0; i < props.length; i++) { var descriptor = props[i]; descriptor.enumerable = descriptor.enumerable || false; descriptor.configurable = true; if (\"value\" in descriptor) descriptor.writable = true; Object.defineProperty(target, descriptor.key, descriptor); } }\n\nfunction _createClass(Constructor, protoProps, staticProps) { if (protoProps) _defineProperties(Constructor.prototype, protoProps); if (staticProps) _defineProperties(Constructor, staticProps); return Constructor; }\n\nvar CaseStudyCard = /*#__PURE__*/function () {\n function CaseStudyCard($el) {\n _classCallCheck(this, CaseStudyCard);\n\n this.onOpen = this.onOpen.bind(this);\n this.onToggle = this.onToggle.bind(this);\n this.$el = $el;\n this.$el.addEventListener('click', this.onOpen);\n this.$el.querySelector('.case-study-card-toggle').addEventListener('click', this.onToggle);\n this.open = false;\n }\n\n _createClass(CaseStudyCard, [{\n key: \"onOpen\",\n value: function onOpen() {\n this.open = true;\n this.$el.classList.toggle('is-open', this.open);\n this.$el.classList.toggle('is-closing', !this.open);\n this.closeOthers();\n }\n }, {\n key: \"onToggle\",\n value: function onToggle(event) {\n event.stopPropagation();\n this.open = !this.$el.classList.contains('is-open');\n this.$el.classList.toggle('is-open', this.open);\n this.$el.classList.toggle('is-closing', !this.open);\n this.closeOthers();\n }\n }, {\n key: \"closeOthers\",\n value: function closeOthers() {\n var _this = this;\n\n if (this.open) {\n document.querySelectorAll('.case-study-card').forEach(function ($el) {\n if (!$el.isSameNode(_this.$el)) {\n $el.classList.toggle('is-closing', $el.classList.contains('is-open'));\n $el.classList.toggle('is-open', false);\n }\n });\n }\n }\n }]);\n\n return CaseStudyCard;\n}();\n\ndocument.querySelectorAll('.case-study-card').forEach(function ($el) {\n return new CaseStudyCard($el);\n});\n\n//# sourceURL=webpack:////Users/cody/Sites/tech.clickhouse/website/src/js/components/case-study-card.js?"); - -/***/ }), - -/***/ "../../website/src/js/main.js": -/*!****************************************************************!*\ - !*** /Users/cody/Sites/tech.clickhouse/website/src/js/main.js ***! - \****************************************************************/ -/*! no exports provided */ -/***/ (function(module, __webpack_exports__, __webpack_require__) { - -"use strict"; -eval("__webpack_require__.r(__webpack_exports__);\n/* harmony import */ var _components_case_study_card__WEBPACK_IMPORTED_MODULE_0__ = __webpack_require__(/*! ./components/case-study-card */ \"../../website/src/js/components/case-study-card.js\");\n/* harmony import */ var _components_case_study_card__WEBPACK_IMPORTED_MODULE_0___default = /*#__PURE__*/__webpack_require__.n(_components_case_study_card__WEBPACK_IMPORTED_MODULE_0__);\n/* harmony import */ var _utilities_equalize_heights__WEBPACK_IMPORTED_MODULE_1__ = __webpack_require__(/*! ./utilities/equalize-heights */ \"../../website/src/js/utilities/equalize-heights.js\");\n/* harmony import */ var _utilities_equalize_heights__WEBPACK_IMPORTED_MODULE_1___default = /*#__PURE__*/__webpack_require__.n(_utilities_equalize_heights__WEBPACK_IMPORTED_MODULE_1__);\n\n\n\n//# sourceURL=webpack:////Users/cody/Sites/tech.clickhouse/website/src/js/main.js?"); - -/***/ }), - -/***/ "../../website/src/js/utilities/equalize-heights.js": -/*!**************************************************************************************!*\ - !*** /Users/cody/Sites/tech.clickhouse/website/src/js/utilities/equalize-heights.js ***! - \**************************************************************************************/ -/*! no static exports found */ -/***/ (function(module, exports) { - -eval("function _slicedToArray(arr, i) { return _arrayWithHoles(arr) || _iterableToArrayLimit(arr, i) || _unsupportedIterableToArray(arr, i) || _nonIterableRest(); }\n\nfunction _nonIterableRest() { throw new TypeError(\"Invalid attempt to destructure non-iterable instance.\\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.\"); }\n\nfunction _unsupportedIterableToArray(o, minLen) { if (!o) return; if (typeof o === \"string\") return _arrayLikeToArray(o, minLen); var n = Object.prototype.toString.call(o).slice(8, -1); if (n === \"Object\" && o.constructor) n = o.constructor.name; if (n === \"Map\" || n === \"Set\") return Array.from(o); if (n === \"Arguments\" || /^(?:Ui|I)nt(?:8|16|32)(?:Clamped)?Array$/.test(n)) return _arrayLikeToArray(o, minLen); }\n\nfunction _arrayLikeToArray(arr, len) { if (len == null || len > arr.length) len = arr.length; for (var i = 0, arr2 = new Array(len); i < len; i++) { arr2[i] = arr[i]; } return arr2; }\n\nfunction _iterableToArrayLimit(arr, i) { var _i = arr == null ? null : typeof Symbol !== \"undefined\" && arr[Symbol.iterator] || arr[\"@@iterator\"]; if (_i == null) return; var _arr = []; var _n = true; var _d = false; var _s, _e; try { for (_i = _i.call(arr); !(_n = (_s = _i.next()).done); _n = true) { _arr.push(_s.value); if (i && _arr.length === i) break; } } catch (err) { _d = true; _e = err; } finally { try { if (!_n && _i[\"return\"] != null) _i[\"return\"](); } finally { if (_d) throw _e; } } return _arr; }\n\nfunction _arrayWithHoles(arr) { if (Array.isArray(arr)) return arr; }\n\nvar $allElements = document.querySelectorAll('[equalize-heights]');\nvar groupedElements = {};\n$allElements.forEach(function ($el) {\n var group = $el.getAttribute('equalize-heights');\n groupedElements[group] = groupedElements[group] || [];\n groupedElements[group].push($el);\n});\n\nfunction resizeElements() {\n Object.entries(groupedElements).forEach(function (_ref) {\n var _ref2 = _slicedToArray(_ref, 2),\n group = _ref2[0],\n $elements = _ref2[1];\n\n $elements.forEach(function ($el) {\n var styles = window.getComputedStyle($el);\n\n if ('none' === styles.getPropertyValue('display')) {\n $el.style.display = 'block';\n }\n\n $el.style.minHeight = 'auto';\n });\n var minHeight = $elements.reduce(function (max, $el) {\n if ($el.offsetHeight > max) {\n max = $el.offsetHeight;\n }\n\n return max;\n }, 0);\n $elements.forEach(function ($el) {\n $el.style.display = null;\n $el.style.minHeight = \"\".concat(minHeight, \"px\");\n });\n });\n}\n\nwindow.addEventListener('resize', resizeElements);\nwindow.addEventListener('orientationchange', resizeElements);\nresizeElements();\n\n//# sourceURL=webpack:////Users/cody/Sites/tech.clickhouse/website/src/js/utilities/equalize-heights.js?"); - -/***/ }), - -/***/ "../../website/src/scss/bootstrap.scss": -/*!*************************************************************************!*\ - !*** /Users/cody/Sites/tech.clickhouse/website/src/scss/bootstrap.scss ***! - \*************************************************************************/ -/*! exports provided: default */ -/***/ (function(module, __webpack_exports__, __webpack_require__) { - -"use strict"; -eval("__webpack_require__.r(__webpack_exports__);\n/* harmony default export */ __webpack_exports__[\"default\"] = (__webpack_require__.p + \"./css//bootstrap.css\");\n\n//# sourceURL=webpack:////Users/cody/Sites/tech.clickhouse/website/src/scss/bootstrap.scss?"); - -/***/ }), - -/***/ "../../website/src/scss/main.scss": -/*!********************************************************************!*\ - !*** /Users/cody/Sites/tech.clickhouse/website/src/scss/main.scss ***! - \********************************************************************/ -/*! exports provided: default */ -/***/ (function(module, __webpack_exports__, __webpack_require__) { - -"use strict"; -eval("__webpack_require__.r(__webpack_exports__);\n/* harmony default export */ __webpack_exports__[\"default\"] = (__webpack_require__.p + \"./css//main.css\");\n\n//# sourceURL=webpack:////Users/cody/Sites/tech.clickhouse/website/src/scss/main.scss?"); - -/***/ }), - -/***/ 0: -/*!*****************************************************************************************************************************************************************************************************!*\ - !*** multi /Users/cody/Sites/tech.clickhouse/website/src/scss/bootstrap.scss /Users/cody/Sites/tech.clickhouse/website/src/scss/main.scss /Users/cody/Sites/tech.clickhouse/website/src/js/main.js ***! - \*****************************************************************************************************************************************************************************************************/ -/*! no static exports found */ -/***/ (function(module, exports, __webpack_require__) { - -eval("__webpack_require__(/*! /Users/cody/Sites/tech.clickhouse/website/src/scss/bootstrap.scss */\"../../website/src/scss/bootstrap.scss\");\n__webpack_require__(/*! /Users/cody/Sites/tech.clickhouse/website/src/scss/main.scss */\"../../website/src/scss/main.scss\");\nmodule.exports = __webpack_require__(/*! /Users/cody/Sites/tech.clickhouse/website/src/js/main.js */\"../../website/src/js/main.js\");\n\n\n//# sourceURL=webpack:///multi_/Users/cody/Sites/tech.clickhouse/website/src/scss/bootstrap.scss_/Users/cody/Sites/tech.clickhouse/website/src/scss/main.scss_/Users/cody/Sites/tech.clickhouse/website/src/js/main.js?"); - -/***/ }) - -/******/ }); \ No newline at end of file +!function(t){var e={};function n(o){if(e[o])return e[o].exports;var r=e[o]={i:o,l:!1,exports:{}};return t[o].call(r.exports,r,r.exports,n),r.l=!0,r.exports}n.m=t,n.c=e,n.d=function(t,e,o){n.o(t,e)||Object.defineProperty(t,e,{enumerable:!0,get:o})},n.r=function(t){"undefined"!=typeof Symbol&&Symbol.toStringTag&&Object.defineProperty(t,Symbol.toStringTag,{value:"Module"}),Object.defineProperty(t,"__esModule",{value:!0})},n.t=function(t,e){if(1&e&&(t=n(t)),8&e)return t;if(4&e&&"object"==typeof t&&t&&t.__esModule)return t;var o=Object.create(null);if(n.r(o),Object.defineProperty(o,"default",{enumerable:!0,value:t}),2&e&&"string"!=typeof t)for(var r in t)n.d(o,r,function(e){return t[e]}.bind(null,r));return o},n.n=function(t){var e=t&&t.__esModule?function(){return t.default}:function(){return t};return n.d(e,"a",e),e},n.o=function(t,e){return Object.prototype.hasOwnProperty.call(t,e)},n.p="",n(n.s=0)}([function(t,e,n){n(1),n(2),n(3),t.exports=n(4)},function(t,e,n){"use strict";n.r(e),e.default=n.p+"./css//bootstrap.css"},function(t,e,n){"use strict";n.r(e),e.default=n.p+"./css//greenhouse.css"},function(t,e,n){"use strict";n.r(e),e.default=n.p+"./css//main.css"},function(t,e,n){"use strict";n.r(e);n(5),n(6),n(7)},function(t,e){function n(t,e){for(var n=0;nt.length)&&(e=t.length);for(var n=0,o=new Array(e);nt&&(t=e.offsetHeight),t}),0);o.forEach((function(t){t.style.display=null,t.style.minHeight="".concat(r,"px")}))}))}r.forEach((function(t){var e=t.getAttribute("equalize-heights");i[e]=i[e]||[],i[e].push(t)})),window.addEventListener("resize",s),window.addEventListener("orientationchange",s),s()},function(t,e){window.addEventListener("load",(function(){if(-1!==window.location.search.indexOf("gh_jid=")){var t=window.scrollY,e=document.querySelector("#jobs").getBoundingClientRect().top;window.scrollTo({left:0,top:t+e}),window.setTimeout((function(){window.scrollTo({left:0,top:t+e-40})}),50)}}))}]); \ No newline at end of file diff --git a/website/src/js/main.js b/website/src/js/main.js index e86e5bda09a..460cbcc4602 100644 --- a/website/src/js/main.js +++ b/website/src/js/main.js @@ -1,2 +1,3 @@ import './components/case-study-card' import './utilities/equalize-heights' +import './utilities/greenhouse' diff --git a/website/src/js/utilities/greenhouse.js b/website/src/js/utilities/greenhouse.js new file mode 100644 index 00000000000..0b87d1a937b --- /dev/null +++ b/website/src/js/utilities/greenhouse.js @@ -0,0 +1,16 @@ +window.addEventListener('load', () => { + if (-1 !== window.location.search.indexOf('gh_jid=')) { + const scrollY = window.scrollY + const offsetTop = document.querySelector('#jobs').getBoundingClientRect().top + window.scrollTo({ + left: 0, + top: scrollY + offsetTop, + }) + window.setTimeout(() => { + window.scrollTo({ + left: 0, + top: scrollY + offsetTop - 40, + }) + }, 50) + } +}) diff --git a/website/src/scss/greenhouse.scss b/website/src/scss/greenhouse.scss new file mode 100644 index 00000000000..710b606fa15 --- /dev/null +++ b/website/src/scss/greenhouse.scss @@ -0,0 +1,27 @@ +#main { + padding-bottom: 0; + padding-top: 0; +} + +#wrapper { + max-width: 1078px; + padding: 0; +} + +body > #wrapper > #main > #wrapper > #logo, +body > #wrapper > #main > #wrapper > h1, +body > #wrapper > #main > #wrapper > #content { + display: none; +} + +body > #wrapper > #main > #wrapper > #board_title { + margin-top: 0; +} + +body > #wrapper > #main > #logo { + margin-top: 80px; +} + +body > #wrapper > #main > :last-child { + margin-bottom: 120px; +} diff --git a/website/templates/careers/greenhouse.html b/website/templates/careers/greenhouse.html new file mode 100644 index 00000000000..e4a4b3aba4f --- /dev/null +++ b/website/templates/careers/greenhouse.html @@ -0,0 +1,8 @@ +
+
+ +
+ + +
+
diff --git a/website/templates/careers/hero.html b/website/templates/careers/hero.html new file mode 100644 index 00000000000..dd4e59aeb3a --- /dev/null +++ b/website/templates/careers/hero.html @@ -0,0 +1,10 @@ +
+
+
+ +

+ {{ _('Careers') }} +

+ +
+
diff --git a/website/templates/careers/overview.html b/website/templates/careers/overview.html new file mode 100644 index 00000000000..1601bf4f4b3 --- /dev/null +++ b/website/templates/careers/overview.html @@ -0,0 +1,11 @@ +
+
+ +

+ ClickHouse is searching  for individuals who are not just knowledgeable about what they do, but want to learn more. Our ideal candidates are thinkers and doers who are not afraid to take on various roles and responsibilities as they grow with the company. If you are looking for a place to build something new, be an agent of change, and have an opportunity to have a significant impact on the company’s success, this is the place for you. +

+ +
+ +
+
diff --git a/website/templates/company/team.html b/website/templates/company/team.html index 8b4c4e26774..cc6bb75e028 100644 --- a/website/templates/company/team.html +++ b/website/templates/company/team.html @@ -322,7 +322,7 @@ From 9ee6e59f979bfd01cbb61241e9e48308c55de467 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Tue, 12 Oct 2021 16:25:24 -0600 Subject: [PATCH 550/950] Update team photos --- website/images/photos/anne-krechmer.jpg | Bin 0 -> 22606 bytes website/images/photos/claire-lucas.jpg | Bin 0 -> 22886 bytes website/images/photos/mihir-gokhale.jpg | Bin 0 -> 25840 bytes website/images/photos/peter-fenton.jpg | Bin 19254 -> 12028 bytes .../{richard-raposa.jpg => rich-raposa.jpg} | Bin website/images/photos/shavoyne-mccowan.jpg | Bin 0 -> 27762 bytes website/templates/company/team.html | 66 ++++++++++++------ 7 files changed, 43 insertions(+), 23 deletions(-) create mode 100644 website/images/photos/anne-krechmer.jpg create mode 100644 website/images/photos/claire-lucas.jpg create mode 100644 website/images/photos/mihir-gokhale.jpg rename website/images/photos/{richard-raposa.jpg => rich-raposa.jpg} (100%) create mode 100644 website/images/photos/shavoyne-mccowan.jpg diff --git a/website/images/photos/anne-krechmer.jpg b/website/images/photos/anne-krechmer.jpg new file mode 100644 index 0000000000000000000000000000000000000000..1333ec4774b3f79a32627c29833097d18582e632 GIT binary patch literal 22606 zcmb4~Q;=p&u&%$hZEM=LZQHhO+qP}nwrykDc2D;-&ip6#jySic)@4RkRIG}uwch8g z{Mq<903b?mwj0RjHS z{+}S=zX&MU&lUg{1PB0(2!i-8t*l7n&28H@3?JZ-ece<*QIex0 zN9Zt9Gh1w1l}K1cCJx>Xnx>fzwprLHHoL7!BPbh2SAP#szN|}*?QQG+H^;iYZESO! z+rS1k_Q@v;c1x( zYlfJ7L1ik~jSpZeQ_-Z83}|#taZ4M&lbe6}#`dcEtN3|=VdTP%&r_qiukfU?+ z4W0mrPg1MG!>{qeZLhX&%eHRY_O>@2pl&W=WR0Mu@6!c=Nfg{^%o|XYlY|>$lAG?q zvLM5BUE@W+XHq#kGuoNg7t7H!%~=_>#;TGM5dIcz?~)x|_^_?py2)+Z1_2e5I_`;N zQf(*>2&(J2EneKg0zV+PppKfSYeFP9wWux+o(yr!sRd>^8&fayn#J+K!K9`pCJV}< zh5_!rq$@7lx;gAoSJ(kpIXVZF%7uXwGRk2~^;XRmCVgXv<~J?%2$03#)*Vh%Ue_~} z7?*cto&VmZZB<;s#aLzimHE&zI@E^N`U6;P+1@s>mAb^_=SA_XYnk3=dO8Uu5p2Mk zJ60qlTMQZ)oHF%b2!cFdXRErF>NZPHbTj;XA z&e%41K%sGTB?_hpxK?terIw&|}?OgpESwSF2_X*P&3^c805~oE2^QTJ6$f# z#B_AvBHhMF7MLZb2iu~@SaDUOSUz)I-ku60SD-*@p&9zPOr(vcE~%sIy!qB`OkeM$ zgCAa0EB7d*NI@Ia7(f}01`Zqr=NY;-put)``}&U+}Q zXw&#$=owsgL1riW@Pa2nkLumPAkQ?4mITJEH~pd=HgVGJQgpF5&o{O~EEsOJ)*m}u zc}?{+M{9Gyu=PwQQ;Nteo(!d?Jl9!F+9{yR8Bp|vtI)(2Y7&+PZ-G$as-l~ymWZdW zPfANI6mf;eXL)|p_qL%pY+#TJIVs3ggK)KY-M*Qd$F%2|G9)FMeKJNcuofj9%3|5h zfG@2|(Uw#2ZL)84cnZj1+bu`v_HzxF8*>LW!%9B7({hF;Xb)SkV|D!iFodAjL+Hho z^_4C=ERuCuO(S4cJSINF;uu;BR*4yc?y+|Ub-+#Z3dkUE;cCg@xZxUzBHrS=2pJ?i z;>8L!xmgv2+t=`D13P13@~VA&p(RZvT+~?!q~}XNlI}!Lhxis(JK;w7K^K8e!r1iG zKzA;~l%&*lDRkkPJS~=L@lpbj$0kLpouzrMpgGP`q9jw&MfqwjZEWXE{KohkscjLWW_)x^N$A5Q)Z~;HcIDP}T`$0Bxozm;s}PJX zRL;AeyQ-grzdmiW?wRDYTIK459cI|%m3SUgqs;J{G_v2}NX43%o|6J6)O-E%r9_Wh zlN%FQBE#cg1R>pvlPZCK)vu@v9Tpkgd}CcS)Zo#T#(h(ampUlHztt~+!_cy<B z>IA1;sPzrG9Om7jVu^}pmod)fS+#H5V}6iifyg|$u1eb%mpH>-2s=oMKhC# zueAa`wDd$M^db-)_VnU#e5QNaB;z$oYCS6-wd_M!9p+0CKECZL^^RxNhmYzes+OAV z8rb0H9hPX3ZO;x9XqTXQb{sCs*>oLFAwZSDk=(M`obFESug>qX-KQ*2G3m_8x~7h= zBEXP{SS87>py>5Jcld%_VO*tzW8JfB1Joa@DPGoz)UaiCk#*$^%gRovz}I0pX(mWN zJk`XNsZ{k&MsroO{Up$;`#$CYj{A7gO}=?ii%YOHG1kI%aj`96Z0SV)j&+HML5Gp!!D((gBsxPA^aTYK7ML5KyQR zo2yj!kYUM!xacO|&-w9?W;#xrP*tyaGF&dl{Po0O!D}UO6Se2Ud8i85aTjh#E8$M)$83Cth>hmOovW^zr)l1#Uc=c( z&qEdQO`UpN#UdlVV(q@uD_kh4LG1Lhc4M&SgUc68QvIQ02WVU0xwDI`tZ?fd#hf0{ zXkH{!SEjpx1KEq){`+ZFrx)|`tiz0|Lr+(bb+kdY2!SvvV_WLtw7HwpT6A5-%TuhN z^NyQJ#$NvvK{VmQRY}lPW7#Y2(bTNJ71sRvqS_9Gerovi3fZIu2wtbfhAvK8i7O+t z#NBC=NW{^NPSuyqV|s5Q;%d~;?cu$y+KM4AG?nY{O(z!jGr3vah z=`k79GWA~>G%a>gRTQIwo!FWe@rGZ81?f zSEUH)b_6npY&&bmf>LO88xXdKIj-8HONsH2EHbk5vT-j@>$Qc|?I!1%36^I)hf7>? zB$!cy;53Tspe4`G*fo8jyXs@(;iTQ3ig_CFUyh$MG3)9s8BI(_{c0MfB-?otIaimO z#$}K{JS9`DPp+EY>Gj*V1UgnzO0wtbxr~xUWfy7H6wZ$XsTjb#y3|4g>N4_czSDIz zWy?vbT!-pgFu?FTx$U7=rs)_k<0$x~f{4m;MnNwwb|Caw9sQJSv^=|L<>cZVn({?r zi3qrGhI>mjn^2_?i9dQ!&E;_oouZWLrCeak*Jq8Gz95m{UAIlAe=4upOH{JJl* z8@Fo4^n{+lC>A$ZoPlrCz^e^+2_$u z6_RIcDVcPxqUw2Y=9{zTTwkzJIq`kd2x%Yh)$TDO)%@heuBC9P6H5V$Y0?73T@zG< ztGzkVy13U8vpv40+PKTMPrP74s$8A3VexEvP;ObX#8JzMQ`$ZL_3i? zd{4~5MIQ<$L(R7P5EyE>l^t@nCvh#cB3{fC@=+LGtsWOsB1plq>c3KQ~~pYj}(sD;u8czPGEcq}>{|tXAJRSN=<)*fVCL#;^?=Sx3ys zv0BH(PflrELBT_qz=2CggMU_%pGtlEaDj^mPcv@L(>vBYGx+p=ouR@GIOY&?=6 zE^|pqYy#HnzItFY`KuT_);x=Wyx!NIi zG%8#iqb2G|<2{shTAsk_$Q^`b#i7$;W-j%TMsi(OTk7MKUaRwjmUpdefQ+tP=ER9= z_!#~G5?pF=U}91d8m-5}aW+bHfd-CN_)EtJXt|G6h2E!|pk=lvdw*LVZ+gtg@hYe> zs)nH1XQB|#oM%$7yymet%0k7y#7~L_^ov>wyj)CiSoh!+dT33noiT~>Bh=BqizZV z@jP&G1j*Pf%-Ca8dtp6O-2+Ew~ytJq{tt(n$k>#ZaU%$X?JfQZy#6|HJ*|KX`}r)>ML zbjRLb5g;v2I(h9VKI~_s*hQ5z7m;s9I>LN0wk$=atsU|`79OnXmBZkXh4mCx>PJQ< zL6SKu<^-Y~>G*M{SEG9@*44O5F3nZG9MwjtDF9k3XV45^899m}Y`xJ=P~6?P9SiFI zVt*P2(iE4O8Ml^kc@7O#V3|#FqiVlqEdXwNYVEU{qGGgp=L4(Ym$tMOR7kyYF{i>0GN^ znsrrw`9tc^tg{}@GoQK9D|Z_=k^$-quL-&L{+R1+oiK6z>Nfa8b~Au zC1GLalt%kP~k~_Y*in_{mtxX5NK8*a<#5iYS#I_BPHT?EN(XPT|VQWH>0&@vPG_p{gaL z9f4qqBY0K)VIMYE#7Z(8N1un^zd?DoH(DJv#M*Kt&chYBh+>&7J~5xk2y#ktxvQzh z@>e3-(U@Q`G`QZq^7flkO_NZi?eDd5fX&PP~Ja zREFZw!+?LI@JaD((7CZkcIVLUFJ9c9R8OS?M$^h%=SBHA&o`^zO;%U;NoU?Q-_hjseCB@>&^?ZS$yTsfunhmoTB&j6?AQ?N-@7%=fV|4!`wXu?Y6u- z;v{72OT*D-Oez&t`;r5jub%(QmDU=08>uREPOYC7?*(M5rz5SV_(FmAWehUNgR?Il zflG3Px%#s+&D_g9!sf8KpU$bjUDi9LvRL#~ORsx|$bV|m(awcEFFEDgicVFvpC|Ul z+yYD07Cl!u2m|z?Z;s*3*-h@AxDKIIm>N$%3Y000PD($XjVp|A-maHPM-zUzrd|`x z?N*tO&ciKReR$1#_AsS&83lSj)-z-MNH~W%0V|8<1jZ7+?V3!^?8oD_452L3)3vsHe zHHIo+@w=zPY2Au*Ir9*pjG|c1pE5b2q{XtYVV2EpO@>^Hb2vRw<&E$r841@lIAI%4 zmD+rMec}X+MU%RniecAOUix^FDrc|;l~0OoX-^ZG8vfW^#1W`tYf1gp zL(wwI)57tavRXm4)S4m|n~lr%kX1mA2n3rbFx;}sd~>T4cjXjKeCutvP?6&^k8=eA6o8DXB zFm@Vk!xKZpaeG}`ER2QNr$yRM*b`N(@%FgHLMg)to6F|I<0Igr=b9zo!stW5>-_x~ z^$;6N(Iz`zZnSlfX5IlhQWqmkPR!~NJD=2AeW0`OKiXjgD8U=dW?v213 z6mff-WYdnxTtwT7a;gPV@VchxBxqp5ieflYU0&%@rM+sZSjnyku-xn{+`WpN@#=!5nRC}X!(iWTLd#-S#%~tAIort3!&+0+LlrEqi-Z2om8yV?O%hJQp zqTCn8W7;1P51%#ckMm9Qi#)1*&8X*9{kr#aL?D9ZfLUW)R}cIJKf9Aublg_wfR4sqZmUqL$RSJzJV*H83$xX1?oPIn6r@=KD zSYZ!KHDXup92RoR2NL{D$;dS!A`M)Hd1F4_vXCFZxv`X7?%?fUl^DjIYUl`+H#@?1 z7q8&5>z#g%s;d&L8eLoeF|Jtf)b@cB3b+3E*5Vl^<9ocqda@Yu&erID=Bmh~UWGBx z=h0T3i1vu4Svk^nYN^#S9g(8>FG0+!PMEjza!0@IjCm*H0G-`7=E1G<QQ~1+AwxY>RfBqJap^Zsza=)qht$9-NyLEQ-y{0j7_y& zyP-cT38z+D3b&={>FI04$2F;WT9+!qbm!5>gDA9{RXMq`j%baf082VUOFf_h7s@E; zK^{t7NO8saBr&;MYl&H1pWs0pSGSX|?DKVE{En#zJN0XQ%xhb|S37E(*jV~gF0kNH z=F<=vKLABEFEDs)>Sze^f?DPZeLNAz(Du!{kGbX%+;{6PM}{g-do!c$zuTebtFR{9 zc+nH)ORWrEEx0rUp-r4+{foACdD-Og1*_E6W;P~lTmN2zf6Z67dmG7Yz414BJGfC4 ze4MINOz1HTrOTNjKJA1HDwM?#f?PRv7WXBEZE^j$ZZH2DmT^1sCULKGTw^6Q+ZZ3O z&v0?7n-rMyqvi2H!JcZJdQqfY31RTB$o=@`>QBASHL_*KdDU)OJ#Q4YXJIA+TZ(q$ zX@hp`8Q5%ER41oX<8qwc$`$MUgp6ob`=yTzx$@6C%PojH))w2};(Rn1Nx3?Hwfs)a zPN;l0JaBSsbiQaNaZvG?e{$*lPgG{rw;cDix^%Y2*iu%zDtOhbdxdC_S3p^a^j0*j zOt5(hldKIL(d>5`{AIf)egGbKX8k_Z2+|C)SjSIUuz%*}m$TaGwxzX~dnI;QvtGdp z=f|DJ^zij;wSZ4O7NVsO1`$;eaATNR88?f?j@ER#<1ksZBgVEXrn%MOQMEugyFtowC}mb&QDW z1WHVR-}NJHUq~~6ADn4Y2T~;>^NPl6Xxj=X&h0d$!(L|F!SZTqi|UunxlBn*u+^t7 zA23zi8d0NM3*5_D`~X~y7a5%CD&~{b%j`dKG(qpGE2&PEFcwLuY_%OG+EwUCbEn|6 zNlZ4dLbU?W|H?kC>gwiiZE=E`knzc)yQxlsGj4Y?jh{dj=o)#mLMnm<${I4^Q}vEa;J7u; z@u8)VSW>->9G#My2u`?mPB=6Rm{qQ>-LW3KYj2Ew`cV69YH@39-{8IU&boTXEoyfX zKlv?~dpIMY+;gv*TA2^OMt0w8Wn!IHuH$m}Qm&jxf2ldeZTbUX*Kgj2aS+%snWkQK zH&>=9w$9KMttxZqhOHX7(UQSH5$%L(2r*f9OO3Hy?uW&}r{0LC$iUO11T~3%z1C?~ z?+CiRo$d(APl_GaR0`}@<5sH`ADlK@5MkX$Vn^Fumo+gXtBs%tTBm(RI#QmgaTN8E z$%LI|nOAXzb<3-K57l*mEfqiJ`wZC{*f|(hXY7yC%|vdFwwhEY3`nrNweCS!tpOkZ z2Ym$oK_4JsP%vN+aFBo2_CLQG2p9l@giORJ2#P|iXvidlsN{%BND`P>$iQp_ChQc{ zIH;V|w21bLrRe_Q@qaiFI6u%gMRnBiwSm`(?(1j?7Fqh`0+d`9Jy*iyMqK`siWTfs z%k0D!ADK#NOZo6`F$j~K0EYe3cvX)AE1&XZMmAY%4@UG*Lfts`pl)Cu8=|^ZhYmJ$ zPAo+x9>&S!o_ay6)Ux41KViPv*EUr7E>c15JltbgLG3WtTE*UBiWS616`$rG0DP5W zR&2FE51!K;Lb(OU{0%OgwgFohV@9@zs_gU$JhmMwf|EbSDhn&Z6d4~yGf;+b3(F8! zX*2IBI3COewzRSj1zU)b5&UwtJhf+7c8+oL6$98wJaQ^(l|@s-C0;2s>Ls<3Hmvua z>Y+uj;#ifpdrF5_Kw#YUy=tLi(7vU9WbL1buYz(_?X+6wz2$If=^V)OHWQIj=}^}q z*^joAD+Re0^s3y?85L@S-xM#f;JglwGZa{XHWT-AK7x=1N5f z>3=+D>@I^RLpgki!h4&)%6#+)s2QCz19mWxyLZKs@#|b#=B~}(0z7hApRSKkk17m# z#}ZnRTTxE9hE!vj3}B@ai|G_)y^RdRv-tOXv%(Q&OvI=Ms&)zVqvhAOt8ZRL@JkU$`&T=0_WEZlLfATCSq!zYT04_sBG z>JtdTh#&1|9KCBxI5w&>Y^BltLtW6P1_e1=W!O3Xh$~|g$@e>rpC^k5X9?+$BXG9h z2;T)tww9gkFYm>w z@A-fJ=&0frJwn`yoWI5jMH|6ITQdqwFa(MOz)%0+)P{SkT=EkI;g<%UZs{d+Bj0bx zv1$8Mr*x?oTqr}0{2IC|Yqvcd(=$)kDtoS69nUl`5BUg<_*pY9+gKif+GJZu<4A8z z2RewGJ@X}z6P3#{SG-bjD~*h}G$bl>>%pi&E7Fhh%}_0$)S0#;wNj}vR|FBowVj$7 zo<5v(S+UyNm>%V2@oQL<3c3<=zSUuMMO#XNxh~Oq{j3xMX(^YqOWzQG;4?u^m$|OY znVM_Ve2F~+=(cxbBP;9Q=+4Ni{V~eF(BIQ=&NyS~rQ%4wlvX|bmrNHK>9ziZ9A*S> z{k?ggi^)Ic{vx_%qH+yP(SekLQN!_BM*c`vFevx+zJXC`?7Mfs_2OlhmvvtlR;x!I zio~XS$jb-+0hkG1_Mmxqb>wwCzXnn{vZApLWzERyICB#yq*hjkqt4cYcx)dAm7}ku zx?$~+3gJ%|^84Mt-Eh8qD9=z1DUbQNamwhZqr|$Gv4n&1{7p+E8}H%i${2Sr;Cc^> zTc3~_BVcIC4rDVtb-`^{W~AP#+cRuv#4ovHzLfv?cZ~L8bL{Eu3TxoO<;eLmhBdq( zWI+cnZN)8xBM8g-HI8Ep13!-|$e!Tigj7ABAGKlUZR~)k3E`Y*%06@*a%lS;RLL_~ zC$>OorBk+a!w&ztO_gUOm)Z?rkxu^@? zcyN7K>DOYcdlR3b7S&THi^D$2TT=#5=l;l0`8oJgw!ZY7Dn3>f;b#a>pj3VkPzXD6 z_yAj44dy!oQ`boj*xrE5scI&|k{@YQPNR$uugi3|DJSbdx%!vsI-&WCh*tLbZgKGp z=#Gu~bIb9{+!hbZxf9XNEa?jhTp&MLJ>OI3_CnkQf0*&SoNVy`XdYOlET4MWB0%2!Ck zqKwE}}4&3Kt z*apg_Pj<_4N}scp>1JjRP;E$C`ZRydfu`y1`?iLIiMF25fx|l(oKZ5sJ7siO9pcz< zWY9hcPB#x-E-F|x>_5g1ul$+TegLRea5gCIW+=!g2UaYah3Y^tjwpg)Zv+lQ;R&e8Gs!}qFdV>Lb4n>JF zu7n57OI`!JKXGATtY9);a3jOYwmxE?=BVHv_4PX~#+=4%{4LOtYp!++ip49Z*r_?Z z!asnS!g0+MypZ{fn;onrg!ckaI%Ua>OiO0;N765A8Yd8W z9tdMvPv?q>VVP5c*9;uRloR$!XT)NA=*neFg%pmMzoO95^Z0vD$)=@#_Yc`hgYn1Owl`D$fajm?86gjwm9any11d6;u@qXN+1G3K|g;uEJ^8G&IkVtodc|D1YlLCaVxoS}l}jycplY>v2g6c95R*cjzgu-3R1npI`H-?xaM zDmbHZo5QrH2nV}6P}$$f<(!5~M!QX%Lgn%=@x!@n(Hi!Nq5K@bjqHIIFehu9k*sb> zU$Lgo%+lTs6}l{p6>2ND*ziEdSo3G*N4k#;`ixaMEDN!d9~j~mwzB=wOy5bz7bQ#jz38&ldxOV)MnM^BHE}kg+>g7p}1%|J1Ci>5qw>1yWP)GPvtZW;@ zG8blWg5jPyF_F_Nuw+&+t?2Kd?m$*Zc)=gETB!;NWQs~f6!#(i4tSUk1g`}D{yX-B zjs9MknqgwgkANd1NKM-L(_7H*8EuT+ik@JkgXWj`o>Ehpeuh68Q5S>_>-acFI>c`uzNUqxJo4m$ z5GhoYGb|_;$&s!3=Tlg@giTkbQl=0t{Qjm#=f=52OO7%6BYa>wQd9u9fKHMDT2+wP zkC)4KBXjkyB}1SGQ~O|=)3zdnD|5JUZ&BNC3bBJ3-h+-z8$xnR$Bdjd=^#P?8i|mB zRjy&vloH19?1Zne!chkXW-nd#uu?6ZhD}%dpFpS$skA-bHpcXm6b+vt5?=0X`9U^j z2n|i~d|+>xw^VMurGmI}N6ObRV(o@M8c<%bA2Nk8en4^wzRUM5)Hlo_zRry)`r4-G zr7aBFLha!+YHZ;jGH_3#wvlsw6z7YDhb!1AkmqFVhbqU7{ehJC0YO$D=9`zD#Y63_ zEwl`1YW)-IfA*z^pHs!pU^{L=;A%`EZiDX#V&nJQJ&&aXuI=Id!oP~O73`7EEVa^f zPjs7mq@r)~6By#&2W)r#nLaN(Gg+ofH4>(&}0_!8kTja=E(F|4SyR>`_n0Aly7TP_A8w2pn zk%s*Fc zwt$$jhL9NF5nI(}u1RNBPxHV48jr^)4)PQzF)PMQcp4idQ-@j(tSR)%VKA znBfd~N6+tR#Iy#csuO`($y-?ns3KYMBgbKx7Sk+7{9mH;Or$F98OOqu$Um9BiZN^kS_=*vaLGMo6kpQ86~1SS&JeGh z5j=(nH*ki*x`jmAPK`M=28La8`Cb}RvdX`6KePt}7k1?gK5Z^BJ=3XxGD%MoG^AS^ z(!-B1daO!mSdgYIeU3Wb2vM?aF(jp<%C?TDdu%hcbp1z>tIoxo8=}Kc1IOHBaCPMt zFCBCwl*Av8o&;De-%{}#26g;Gr2oY1o}Jv&p2FA})jJ~ZKBdh3vVc2LZI|RP`3r&h zr$WDQz6~7w;1u$YMr6vIm){{nV_M3}t(_5R4Xsb1V%cM?^eNaO*RBzhiIU4lB{L4j>hOSORt$~X*U`4cInD2L9M8PWo*%kT2oGZFnu+hVti>p z`hj(A+dPFOCkS4V=$`0f6yM;{o*I9M_eD6RWSo_FM9j)NQ>tv7V&JXilq?nWWK{+h zu8t)*yy|lgGymPF3nz#UVVFKPFwEstX2h9H>s6Jo@eWM5a*o?=idELebmw^9fK*E4 zIXOjZ*w&17WB)N)(HBkhplqCkh7~KKM--!du3Lsjl!mC)G(f4Ue(NJMD`8Rdk_o9{ z&(O6~UYbUMw}>D}`Zd-TGK9Vw?cmgfgW%tKub}jm%!o+DqDFKtOmW%V*)wtnAYOt4 zj`mBuaHoSE?=Klb8A7n!vu92MQYsU@;(Qr$Z!ANs%+=TEV54xhXkB)KpM!ZtV+`#| zV+YmP`Ye5mNsh1TGmV^)CNV2j&>b22EN|>lh#g{STcETbKznKZl(DmPfc>!KYnU(X zai&EC+nTE2Ag7c@n>9TDeHf?|ukA<{>>V=j4+x~8R*sQ=YhdCLYrdw7j6bIxbHy`b zpuYQ5GZbm~FyfX`n~>^Jj=gGl&;2p%`}mVH)Vj{ z7)~=8*V~-oEtR^sG4Lz)NVPb|)q`f0SC|-z;U9*Q7riCt%FEuijPd`-Bc&QKqeXOE#7g^uRQV6{ zCldUfA_s}T2vI4H%ACb=iaLK|n|iRK*6vaE?@lYju3oe1p;J2)? zJEL5)sb=Z4OuL6)pr5TNm#`pTm{M^C)lyOhahsTi!^e3p)v3qMUYaqhg(H!=W?^4K z$Bq>_VJkxD3`1r^?A#D3fSW7hwi#O#n_!BT9NjhsF6^SySLRF=P0JHBQiQ06D?F&m z_2t)anvpShtVZzIzXDV+60L?ML+AC4X$BHIQ}ofJfEgN}Vt;a;ynzVw|HH$?i2r(= z{%@Dlf4xrsH9-LZkp8h@MMFo#e@M7-@P97?@*_rL{Au{6zc(EMhSPH2snW4J1Vmp` ze3BR%%2qEWlG=sQLd~5`kg$|)D__rD0zlImx)d-KkX8`IR}9JgQ1sOvD?4o3E(i{P zJ4la^wNi$Zqh65hdurX=gJ+a2WpzUf^>?LR^gK-WQP(J+gog8y7C%AScfzCvuL_X8nw}MI#@x( zc={0EDR%n%^Y};B#`{3@20`=9AsSH&uR>%}Uhv_r?*c;Nd8R;TNDy}UVmiCFta0_L z+KRr{8=|-ytslS~1Zjc}0ha2trEbJl2i>kH>{^;7Xb>&iHEq!N9j#c)LbD8@&N4h3p)YI-JczdUiF zL2}U$H@oO6#c2z6aCDy3>M@8RW}+4b*NV{gW7#ycIJ}UC+e1OiSC`^9bMojEhwooF zoME;iw)S(MFEZBJwq5*0g;M84_=r?ka}cjpW$I5k#1Y^5K7hJi);XHUcwo_HcsV9|{cb#vbkfkycsPawfH z6TYE%coM~`c9ao@AAqySFSLFTBK?c(GAc?gb~;8FBJ8LW%RYIBl!{zpCROf2?kgT+heGW5S825cXvg*VX+YRx_b(w?y9zgQ)rqD=fFP+B z!lss7rBq){%}VG*NX>>SC}Mz0ioan#U7QSMV-}K;PN5KsXbK$ZVBX3J@#r_G6S!#8 zzUcue+LJNFzn#CXXBmDG7?gGc=pj}hxlrxkR}aECHp{yjanM#ggbZ`hs+hZJ^d^0Z z3;;(adMxop!b1e4`A4|&dZHH?A|Jg`2zai`$C!{_tTYq-0Muiu>|vH6Yj5-DC&Gkk zZb>Y{vJRf5=O7k>aCl2)He$4~{6=&c&SNl*Xjf7Z@kCOJkF}a~V^!M3m@+E>iR_>v zrG%tmnvK7d9+4lNK68|TpcdFCqU(Up6;gjoaEJ-*$`V%bjSxgL3#=fLH<<=WA2QJ; zsTi%|;A+y-)%_wcNP?`%!X-t8Q=NDq-g1g<^vQheMhT5-p`|e>F^X=F<}zVP9lYR^ zRyWyk(ER&!txL+U;Tr*_4?n#4sGw>OmWVCsq&tg`ZwcA)gg6A=r22@NO{f#t=d2jJ zgu)^yq&c2+y4%7@N)i;RnDW^UlTG~ZN9q(g<}Bb1bLUOw?>;J?=b6Z1=qU{8y&qK99MVynU( zHWb;ZGS9!BQ3c*@PawJLruAXHUk45$=6f@VD76FJ2DF`5<+%2-Q^NzlcYTe5eiIt3 zN>$Yht&BR^(5=%46Q1PBpd#%zbIk)DwvfenVE5{NiRg)ft zr@eE?>R%fZ;fQ7jbpgO8{jDIR=@ zy4eJCb3Ba`SLtcC!-93@crk#K4~d}-j={|Bl|NhV)X&WgrH!KHf8jAOK?qMCNo!uc zdX97#3|`0;( z?2Fu(qn`3WGA&b45aXx3z#BfrO`mqi2kC zd?g8U^`&9mmZzWemeihvyhkiDRRw=ndfkofIKlk&7f2z`HQgMv?KyuLW}DUSsT_=g(DP@1H}3lDTU^Deh3{& zrF1=_pAJHf5aCP?)ftse?Cgm0?i;R5qo|!_1UCDIJNod za@h-0uS%L0QZFw?W88zv0VKH=4gF)$pTwXCcE2TCM{*{xc^kMdgAn&5|A{?0N>NAf z?FM=%mrdnZk5p%-B1bEXiv$P|FzE9{Pyq;DeHkE(k+(KH4&G&r8Rp4XXYp?kL=wB| zk(?w-qD*bQF|coh$=p$p5f!rleTlz#YW^1lSuPk!`c;i%8Oqqlqc|G~O$qzGP6ydL z4X6$BZL6g5NMjF(wCQPx8?{E;=h3xpnF%bZT53oFdAQQAtSWRT2#p&#!`dE3(CnL) z=81Obn8*r<>RtEZ&>FuBmw}Dg+8oKpFG3<)CgGwDArdxN*e_7sC)^ePPXL}5Vd$lx z3os?oj)XMs2X#C0x*8S=36pKlTft z4cx>!PoHCW*jUUtR3|zvR+KnW2wD9M7~dEIZVq7E4Ne{gs|5rjJi}@lA_ure$MDB` z;U!jWG?6Y`{sBC*0ngG<)2s3vp8EVk6TnqqG;GnH#?{<;&cYj-8eiCv3H;Dls_) zihm4&gnYChO}?uXq_hP?++|Hq`{{vH!3Fkk z@B=-#1LDkri)2Y72SvD;i!FPV!;MqPIx-h({x8UsC99z{*wV7LVJ*0?u~d5IQJ9BH z=zsu*;7XL&lnH6>QdqH+b76ZAEDV*8NXlJQ1$nx%&}7OpS+}A7%o}gk#pR0{5xVH( z0GI-0_)GbHTu-r{h9&AwhZj0%;QKWU-^8GITRzIMRIap2U$9ekV9qlO{9qS0F4M`Q z7fUK|sLSSUoB5%ir@3V6hq$Nn0=JVtlU+;x99k(l-K-ipm~|~jqBX9xUnxY5q?o`z;2k6KF@Sv>xl@6kh9PIbgExh}1Z}utH!5QPo)3MyFrCfDwaR z&^BI`cuLqyaL}?AlB}bxlmY1-Bxvc1Kcw1AV%~M6^uiCTEWjv?f#r&#%xl=h@aOV+Ozsmk zYXO@)@!qvfwirBL*oD0Hua#&bEpSR}C0?lc1IztqJp z+OgWA{eahaAMCCBG^vu;A5F@64DkP<^&NuD zu+PZU)kEY%ycC%lDY!$+J0cM)Q0$gDa|`j6AnXUvA&k*;A{l4}_8IJ<=^WSjz{VTd z!9kUA*I5}!D2;xJPp4{CJ6sl9jWTLy$!--QnPJ(~^_0->WZ>Af{D$7zHTyZ4BRn#v zV`V@Jn7^=CRl(K*w%KazE0`!O6;}|Og^5au%P5}+AYMq7%Dr7ml1+jr=h={`1466F zh%{(FBVS0N4#=tM8~F%pgfH0bqGfVZHJ2`WemoRXVCD0hiG zpTUUXEX9LN2E~?WoJ^zjP?r+!kL=_@k_ijEA4iFMRH0ttq0=HY1#Fu3FhICyS>NRL zB9G4|QHwm&VCp2Z7aG@X2C)8@lV9SI%g1#ORhS~6+QgXcRPPPpy%ns)?-W{W5Epq! z6*UMI!V)+};_0GQQP*M6aj~hW!D|%|J%o`i+;_ zAzDx02yg z!8jRdb@Cv+Y!qBn(B{Bl2rz*E00vN>ArPTdH^v~g7Cw=eWA$lJfE=41-bL*Rk!=XU zdows812*FLgq|3Xn;NL2=_zzfgF82+5G!CBBve+Mnt}fS zH3`zkC{#7fVN#>4cFWuIV5IgLt0O7{9BQtI{AMo~6Fz^sjow5`>6B_EYuvneNE{on z4P7Ex2|Qs92$7;fL%Xu&;!Jfmdq2#Gb|9+l+iMEZwRH*jhMLSXXNf zFr75fiOm_58g>1}aj{TfRmz%83W5^RTbt)2#;OCCtx_X(S4QJ+J5vD{vtA{nEbLMY zDeXCwwd88s0;J~-?j3hS2+us-mwLf|3_`6k{H_YuTE0cbtO!h*)%2EdP^V;})E1pb zkASbwW|N_>i7JAaQe5rM0UgJ5=2cD0z&=Y8Prs7aoN8vm-9)AuceH^VM-YGBL^HTz zHGqAMMI8?=IUGpwo?Rf@quad9F#tv}EWnu?MfGE;E9nhAn>dCzZBoB3SencSk8=Wu3)}>1=9AADT`RIaK11n%{PdK$ zoe&KyvLx|%jaQ~Z+EtO>qmRfS0mhgpdQ$7iD_eqI)g>AXw1k%7#?S1{sX+_C3X98H zN`@#}b0GDcLd2I93~UTnaF3fc!BUogwlJ!xP;wz|6~?g>gL_-RdllW4QG!(I^^9`t ztJ#o656h~9oGs0YTFdhzdo@q9q=eE}hOrq)I4l7TE#AvfjWGN~r_$p6o2r#-nLFZS z>oNfaih9IHV(L1E-kC$;7eotO9ZEM#j26Q9Trk$vd>}|L$aUE&r*g*&tP*n9o zvUCkW>42?UO^PK!f)wU1L8>4K1qQtG#t&Kf3yy__ zZP@lOQVm?Y8SaRkt4OQ`rSX-Ls)Ok%NZr{AtMs0oB*RBE_H~61jG>+E0gi^|IxP&QH4>91@h3qm0F%!*jG7Cu}LQwpTZ#^Tl|A=QJTY_^dMc0fSV^*beS zIR}B*!{D5>e^8>Y7vmck6(>X~d*&E!%6+8xz`^Q2_C%UlK@Zp_kd}b2u#Z9o!UQ$1 zumPc|D(5vGK{mGv)$HEYa za`jV~wPB-JH1NNju%5%uuWJa+XFjH^ZL*#!M0)f^J}MeIW3bx?Nx_#YfG4m8Q>o?q z05i>=&lzx0(&+SII&XU%1zYRTfNfUTcBOoUgQQ+mU~v#~+$W+AJw)J@NZ%DdWza4p z=A$~*KK zR$p6S`9~ec^Qk$05RSfRrr;%|nk3Hd0b{XPs;4~HlJ)^mdca!L5330+Os*iK!HaEW zKG^*wheRayDB}fH2uBM#%mwD-1qIf}v#8u51K?}ONDA*R$HoiTOc>e$opJnZf zI}}U#LM$V^vvfoM!~h-<0RaI40RaI40RaI3000000RRFK5Cak*FcSp-+5iXv0s#R6 z0RI4P^_aW-zIU^9)x@6<^zW+a*{{>1X3aC{?*bW^e_!UV-Ts|i;XZWN&r9j)F&fsC zx&B($?VGm)#P^~-h`Hi3qT&8p)2>{69`<~Hfju?*&aT?Y5PycW>&5GEIp=?)UVc9V z&}U2YPw>{YqC8In-AXU4F>rOC$P0cyBtO#qHXgsvd-_jbdxt6q5d%l(^Uv$)qxkP} zKg$oVog&MRSGW`L{c}#DT>D^|P+A{Aw=zbi9aq&5eH>Upx98J@4vOu(t=p#y##|;E_K* zx6{R0V=3eO2-suzead(t+mG`60e({*AQ@i@)GA#CZ1 zpXVMo&DM>6#%B60q_q(jUlQfcMcnv#r10=fFMFphJzt``epdX95d#%lF_uCLEuf7N^GgTeWHVzdIQD znzgq))L2dH>hD@-!otMNGZ)K8g2#>YcIw;ner|W4uD7SQqCDpjo0+lcG2ti4`Mv3B zEniz}YWbK?JIJ(jONptOi{^5}sPLQSsmc-qO^XB4m4BK@!tax}VN1WDwKcC6-aL>4y@$i^i56?l>)3uvd z%;qL0EWi$b4tZ}==RQW8Ojua87tX?^t4nHcpP02`e8+?ObiG|Sdwg@Ww~-jS&GSuh z*5>Y}#2GXUnN1Ix(yMou%$FAITDoh-%kzVF<|a(Ud8_B9;p=gkm!b2|)v#Zu$k?1&wpP9IL3c=Aro7^F}QA%hjJxG~X=b z*`sC@PUlaZ#_dGQuRbP{VENonrcXG*0P!+n)uU%~YaA@;{vRLXB0RRI500000000015D_6TAR<5#0#GtSQegku00;pB0Ruk({{X1}06?j31`qh} z&FOu9AIM5q+0!MLTj4Z$>DoEq$B3SG1rE0Y#2RHbKB<+R5n`F-H&V| z{w(%Dlw+nwygrJCNWv)CYjX**75NJS4_l|znu4Uv8zF-%H4~oJ$khzNgUEv+5G!1g zYLTcQL<{7B8qP*^YmD8fG;qPvc_)DSB@gmuxbGwUS0Us*+TdI|;6@3#i|6Kv@A4lw z5+D|E4a46uVov*D1r3{DC!1C#IL<;dukrvq5Q@G}4zE<5>==`tO%k=8*`{(E?vEBM zbBCUKHH=x!8g}T?8Z_ybG@J-Mobm>_u4|6C)*9xxM^_rcagLASjOQUa;Wf@MiK04V zEf{Lihps@zYlUD3&(F(Nx=2T4sK#98lEfZ!#Q@P>R0y3QTDGqknsiHTENu`yV~oad zDUN-M4Gg_(az2Ra!g4x2(^UzMma?#c%@HkI5z*n%656slXLbYtp%~~PvUQiLMiS73 zP1t4NYp05U6BJ{v8#UNlmm*7Oh?=o@u$Q$VvSR|JrYQLAAThVs-E!K|meUO}(=rJ< zfz1+-l$My{^mrTvv%(c9;wsBRJA7QR$-}`YQc$s3Xi7jp<6c_MV?A@1O9>qj@o1cM zc%ed)lEFT0V>N8SBP}z`b*={nDg~=ygk>Y*#iJcFx}*w99A)!lan}`L0@}4}gg8Pa z%LHMz5t$Eof)bjr;gvSUJmW(26jK=E6C>|M>ywM#njJVFuLSvMagmf%Ua?=)wjX({h;%?B4 zHIQSjkSG$GUhAJMw4^5qZHuN_Tf-?G6D(s0grp@VIO#@`k(Xs9Cws2(P?lr1}HSS7njmB*a@7yZxfwdwio zSTLE|mX!gBXjzbVNl=8Wv}L6yN4A)4oAE!77vjbEDfsVuJMMd9mYCR#9 z2CcZ+@i7r^?Ff9l|F^r<>_QQUs0G!>%V;OfFMF*Cy z()HzKBGW`m#!_{c0~SL-MdW4KQtjm$Iu=+SC5se>sj5b1hJ}`l#^YD-=x0{Uq{!N1 zNmUIZ4CS$q(Fsjvgk@G-l$flJhwJLf_LAz7h?2!DL~Gp;j*SxfWhZYKIIY)x?X1cR zeNs@#G1(G{5viajK-(Nv|GC@*c|Jncu0RsU8KLGcSRZmLQN-B2LU9J96 zy;OU4^!VtHSpK8OOKEPj>0`^N{mInB_S<9=+lqM@-1WrEYxba>EoMCk)rppu7LQi! zL6sLmAc6=YkJy7MA_zUoD$!hBkY%zA�>sB)xL7xW@(%9GH7L=j1>k#p{i&vrLP z=(Tz;rU~@$PieNnJ(ed&8(^K8ZgEDUsI){KZN{fh<_v6$U2c@@#^@GMdB?VMA;R7 zqYrFY*Sa1(W?DXnx%EbttWrikx}NNAeVEymjk0&OvC)C)k@XuycG|K?)N!{ujbv4a POQz``DtM4MsdxX``Vqul literal 0 HcmV?d00001 diff --git a/website/images/photos/claire-lucas.jpg b/website/images/photos/claire-lucas.jpg new file mode 100644 index 0000000000000000000000000000000000000000..4ee456304965ed5fe23b850ba66faa822da05641 GIT binary patch literal 22886 zcmb5Vb8sbH^zVIQTN7(y+n9-FVp|jI#I}tIPK=3t;uB45PbRi)JM-qb_jm96{`;=3 z>aN=O)T-LMcdz|jyZEDUUHEDS6x9DEW299$wiEUZtIpNL3F$;rua2&g_&k$om1BPaV00`<`q z0UiMz5fPmX7Ymo{|F`|?1z;gUi9?CQKw$x(v7lhEp#Jp(NB{t67^n{e{9l8Ig+qWr zg#NFV=%f6<+W%YkUu#$>IQW0d08|(#05m2H=11MV_Wx-B^&OwaAWH;PJN$o2{^tpW z&m14)9Q(lbRaWJUCEKdAMs-8hoOj4=S$PIr;T!n>ZBE#Ma}!UhY~9t2NBSijSYXti zXt(<)JDE1628<|&>`b%%f17I!9K?DY=x{fym>hU#@%0Qk?X7D}Nb!$^=9JY9vifaL z+&yRci|9W8zn!2nMXo2&l${OC4KhT-2uyD;kfeEPTS`XkXOT%cRCsl{*|qpkTvTle z=gCqD^eX?abEx;_PkQjBvQ_kT$df}Qvj*&Wix3U7CCx5T@wmVF=IrJCD#z6cXzhkN zO$L%@I&a@ky#AW~pYEvjS;FlH$9}O@Wli7t?{snw!**GJRLXcx?YmuXzwowKu*I?% z88UTEgaP%~@P_HINVt8nFE^SA7#`EkTh~ngzZr05KHCX& z4*p8tYbr%fL|lw&WX{M!z(D8I*evU0djGMmpLyz&3zr8(XCWGqDmus4=`S*Kv+7n6 zNxjda~bBfZPM*i_z!>Y|;uU zw)Yka?{0Ck9%7_4qo>U?`*YYa<@Auu@p#RrD`2tel^F}Tbear|@#}wi#ICRP}4OF-vsPUi-llynabq{f~ueB+_Yj!^bY=$c^HZSykmRW}X{qsd<>p%BN-8;jSZX1m3Woz;8GHI()C0 z%&$CvlT%*ejqc65qhK(tSSuHUuW^=OJJ#^O#t>%M-+q}n?W^4PT@4?4H~m&y=K)ra zH6U^2S)pKUTf?_fH6B0H8F?E^4SZYI_J$5;<_>U49MYZhUMhxc^VhE5dKV%Q=a1+3 z{U5GL&?VYgWAIE`%Z`-zDCO=Q^S7G(bIe6tyAC!(Mzs><4fv4K$L8_ zL34$-@QGR$aYw`6Z63pn6Hv^_cL|XOrrry9Yxkr^9>XNCJd3E*7T~mpZlPi!_5C1@ zan)r7eRQ2~_s+8B-~$KC|1d}{aNKiRLF-l`?-whez}F!?ht|wiaI%J~r7?&v8R0b) zqYqx!)F0HxjQg})%%*lee0$7H+m(|rg-y;iSy|gkEBNxuTf5`O>C=B6M&xeLc#0OR zXRg@fTt={V?YE7r>P;6~PaSr7NWFpFlk1U|97=QS)=o5@<~eO55EecUl(dVuxk<-Y ztK-U$rV7vHbg@*jPELsIEZ_JsoOu2E_(1+No!;9J0`}*>XslW_BzUhN*;e=_9!uCw z9CqCpOn|=ATh&Ko3#n&}R5(?Kf zf~tZ4eRU9Tmv-~0|1>!>(~V~7*n-xw=hVIa0W9qpZ@H1Ix+J1i%FeuPFHg4f zCwzA+)RQCgW}x@k_}_$~&obH6OUGxW==FXA$QcrG*}Ii`z3xkjZ8U^1qQ#PAg|MsM zkc8qZV)y@s2Y@c4%53O;)xj{U*jbt0F27tHcevWO)DW67Po=n`C-h@UXDkowl0Y&4 z2qvwJQ_wY^jO45liT6X!?d**fP_lgvz^MxS06{(aXUB%n%)Z;Vr06YY7)qqOS7TJO zuzF;3kj@{?TrZn?i0ZnJ`EW|?{sYh?14`$Y>V+)plT?axaMdNmi+W2A=En-2AOCX- zAgWY=>T^KOWtM&Kr)CcJUmWW6Uz`7Q@|W5o$Q((xvA}kGLl_P7gk|YuhtZnr(7|@# zkb8SVf>1pJtu@=x$}KEFbD`J|BXy-B@IM~LPfmTl{d8B!-3x2bxtClF!3uM|`!jEj zplf;6G>rmTW1N7UP{ePVb(?d$z0CbK;f?Gy?brtJAgb+I>KS8OCHHiZk0-a^lK6i( zD6$Omb^FZan*3zh@ygh${8dXWr=ynH)q+xlSJO3flG zt-dr1H*-<{9F;giKP|Fs8p~X#g0^L$yVQMtx6($Pja>Lce%UyQg+5mC4-#CN$CRN` zzO9mN)qc&WDCxx@ig_OQQnrhl2C;5c_42VkZQRm*R*gaSOQ_EvqsDj6R&)1h6^%AB zxtyb#;!C$wGli=Rh0Ke}176^-51>G<$&Gv8h=Z2So}LuS$b6cR@JeC(;CHY48VTuA zQyEO2-M=hgOFQPe7-?OQ*Y-`uiKR{AgiH4N&r|K_yq|JJpv z3;m#S`p$2PS`2~j6Ov{dWUiTnejKdHp}QZ)G}w3?s-M`$cG4M$W}l}?8`)*Bx}P!? z_RB*QhP*|itEOPRGIAtrc} zN@8xjc%FZVp_eDc_2(y??>A+B=hbaDjA}IUA%qa9#f7C@%JOJBC5U1_?LHwN<>$wp zjJ~Z__61WxGGk9kx z%XUQ|h(O~??+A%@u`!2U-rn+Y&8wtbwvXNjTS?-gIiKcQDW`*con}wzL3l*4Gq+{= zt<~s*j0&XMLjmnxT$nHV8PUBfDas)3&;HH+%6BwWmwXhUJm>M}0|)7O#;V%5j_EA) z8}L{b(+mG#_nSb!hd{&MEi;XeSaGq**>ZNpyLND{V58c~zQ z9IyFDSkHqAfQE*Kf`NgCgNBEK`t6e8670H}gn3?n1ah8>ux`qrCZZZF1!}EknacT{=Wo;b~t#CG=)3YjZ zR-BpzR^cp`;q=!r6Y%>?VxOZ6VKllxwfamcQ2yFCAE;3AJH zS?yihSk8WC$$9Rk$KiWfTr8Z>#tIOk!Q+NJ(OI&ww661H<2K&rVlp{og;>K_?Eeq2 z0n4H-fb<^dLNc(%)ea`JQ*FMT`nk~l4?y{YxkQOnUz|*0b>u3l6`(Ac^AA8CuiBFO zDosnOU$InyAluI-sOIpdnqct{U_ED`q$Uf-YFIjLhHOE2yge^)EG->oh5pVpmb(aj znXS>0HaERv&E@%W_jaArF2+XkbwyNa?VJC0bn(y@UhJ>Xn&T+>B;s$BPdJ~qz3>}X z22ow{pAw>881vG`+uUDx=JTHn#=N1%-p9a#v95DI4BzXT8XfAs$jL{SJJ?m~NZOQ| zaDfXlTXped)5?GsYE4b6Wg1T# zD>!mj6r749IA;3&o4a+Nujz&v5i^A?kUwiP4*XdmTWyMaA1a58&e|$YkMIG#vP4q2 z_w;4-q2%c$z(fc)nMgt|J)N=AQ>g{<4=f--eg++TwoPY{wSm6+1KkEy*#JHs z76TQxs(CHJ;Tw=j^y%mQqrVqi*PE}to*SuyOJ6&eHsAw}3&nOq2-mrOegq!(gRx4T zoECI+tTK+wf2K%tLN{aM7NgMDqq|2UQa)j4k%ZJKdlW6x{z2`L@qEsUTA|->!?ffj zzpmhd8ZT#*o1p;R^JVij=|Sp(SypY2N}TKpgBkf{X&F%EE|XHnEB)R zuby`I${zOwE-!xSj5RfFBXndTe&&wFK5fC$?Hv>N`JGKr^P+k-^iBFxwVCO$e@uxqwVazFR4sGQ!JeA6A2_)x1}u2SREJ2YO>{cleeC~q_D)GK1I>OR};(i9O1 zcK-n3NJ^}WPNeNysN*}qF7f>S0*;=#@aBEn-NK?{OU&1!1)b-6gmqZXMa<`f2_@G4 zq{};Ap!ZVMPOLK5{eksLV{;zr-YDbwq#K^Pikidnr|1!16S_*`RmK_gSgkDIg}tO; ztrqN&apqVO)TlGisK9;iaN?&Y35w34MIY0SgU_Z<4kU}Qn5If&GP(Ls3-AaU{wCIT zDd9a4zDv~(I3$NLtGvfa&jC@*199?}Kb$yx}%s;bM5WYzdIH&0=sW)QmNmO7uJ>Bu**(K0;`3y27kFukiam1yQQSY-F z&fi;$dk7h5V~!YT-k}e4QK(d)&hfTZ8m86D*7 zTQSQ!^XwYp^vsuUdHvJ2d550kA5Ullqtz8A9P!4cLB$%CV__4UNxr}GH8Q=-v=)wa@|RJXOT%LxxXm`66oGZiPA9l6Jqy)4c>LwTe^W6p!0C-;0FYEQcf zAUo7o4u`bD6F;^Ares_!Fn}MNT(7(P)4%+IS>bax+7z0gKh^%|rjkFnux1gCQWo6S z9Rz%8w|?M!FK)K3$hB$v+F3X4>>XSQG)$u>o~Tg4 zHOn(X*CR%WVQU~eUX-!f%s2H3@ddKJ@^jlNgq`|saE8EF*DIs^`frM$~w7@H;Ynzn-Wy}pf&i|wP={j^! zkn*QgUf9%G10?uaM0TsG1L6b(mG|Q)x3gMSIIXHL-_8=0rDnrr2J_4u*%-C)5+2#u ztqqkesdJ-*sU3hgVSRi%8q$Y4wHrPQOVzzjSUp?>t;(RgJl$9Iyp>$SjDUaZs*5$g zW2HspIHP*F%4xDR@bZMu*@CfC1C7Z8mfv@A%mX5dcaF^*p9zQ&D;Drx{;;rNdRoJ- zw2q!1wg?&M7UMl)M&cG_I@M`R)6C@FEkb3fJD720XeiCLMMza{qKn?hEJql2QHTtiD$9wMF+aNYRD>)`)KA@d01PdI-TO zRtFPTk#-{>C>Hj8iKLFS^WtskF_2xhgs|+X@$ru8Ta|5pTX;D#rhq|S@}^jCqsJ0Q z@vRu1)|7ZuhU-L4j!lf!ja#K%?S(wvFJ{;>JFxm8=(he5)%@77A_(wKRn#T^I!izk zWUll05ctq%*X=&U_xoyy1g1T$y!wR`6b0|zZ-+ZP*S9sxGnDsn#BlFAnrf`9;*rBlM-NEr)Z*#i zDf~rG)yey-tSX!FO6v69T4*oMh8LPDpW;CQ*HYq2{hv)E?65QEJx#Jk{4V!}%nQ#p z1Q&a2GsMw|4G7YU`*0qmk8lC? zA|0;C6(jV^V;Uh!{G#ba*oM$+gIR`2r{Ib39O>eeowf5s+vt#LW1+riYH|z@ef09T zy{~1EzMAHe3S?`ksAp9U8k~20Rg5q6_pZocgLFL6xlE|5GS+qcsm{8@_y-`vrGiN5 z(fi+NYxw-~J)SR;T!DL)yvFbS_gS@0=b8=kZ12ifV)bc* zD^7dR0%8p*CkP|nbAOfPqm#1`cO;omwXQSE8GZojpq?YRV7N#Xen{Kel(DP#r=Nv> zW6V~~cu!O>z0kiY%fS_lBA{BZC?gY`oX6yU3rNcYf^6{&_T5Xsg%00%c?@QFEFQ%* zkKFJEna^%d3%VJUjxxteY2f~D4H(=DR%;k(m=%NdG>WAs0(X;K~)Xw0Q5P z%A$3R%21V{e|M{!2q~S#l98jOIW>FZe-G$A3!4KgTCA^-8;`#x_IsWZ<4DSVA|nNA znEV6$Im?UenG8l2q&6ic<*&LOztq1<^ueHCn@2%OH3u35P`KC45oF2rA>qqkTFtu(HsXpw^lDaPnot&OQ6s(X#wqA}4g>eK-o6-T5ZzZB*1a6f zof}c!C&XYxj%?wkB%@UDI)*+<=qTbIwmpx(86zFh#Ziq;ZkBT%t@yJ-iK$ia`X`dX zeR-TU7%-%~$Q(}XtFkDk)-7KYF~$qWD_jc5M;h#!JFCXrP!F5$zI~fgld{azy5G%q zbtXx~PVlOMxsHz8+DJO3rR(G!t{3IZHQ#`##juJ@zn=&Z9gWKw(5M1hgjeTuY%j0f z$v3rg#w1xSBc=ZOb@%s+6IVAV6X9c-wAHb&yq$Pfq=!pXyzbArj+OCoZ%N(i0(HBl z?2b~oXX}0eBaggT+;9Jh(L+5lrZ#=KfEj9Yu+L+t8ZlH4KnU87M&uaHea)95eS1N$ ze}X2&8mlTNiPIjZ7Gj{f^J;AsXJ z2G1z?lS@%Byd(M_qjV8&UlNVIxsudHWJVbR>;^AIBKAAql=Z8^fMewVnVJW()Y4y4 zpCL0_DmJ~x6}TPdYOOZFK(=pM8je4q&81aya~spIR&@Fj)4Mh zA@ahNQ6>v3%v$&eqjkKAFL9tIZHRk9qz->9lf+7$PA3ZCnTe6xHPh(ZP-?sA*i9(g;K#jrt8e)bYzCtxp-<>zj%*6iHnvn;l@? zBa~977=MvJsezpu884-EdXc}Pok)*KlUI*6#Z|Nj`)}L9dwrjI)(e+Ozoy3Km1>x! z(zxfZ?*aDwAK`(%DH?~EiKH2muqPCz&Jf!;^172zq6|&2^LZ%)W#@lk6-Ki%s^4;w-?Oi?{h}`zc-~!O+Dtb>YKQrv}PhwAkE!& zPUso(<#6VyC7zBG+7CbQ?GuK|X%xhZ5{4xNl>dh7Ejr3GcE~!HGqEx5t`7-bro0#{ zY9_KS4N;i*JkY#0gdQaD-@oA7S#}NaQul9pdAyBT@?MP&^OC4-t4#FVEvH+!y1!OB z7n)0Ldm!Rp`U@>aI*~|+W#oLU7|%UC0V9n|YV!*>daD!F`Y{#^}Q3#dwRf5`YJ>qKxM79Uy1oaLuVkiokSAn z#`|ofs6n%TfCxMKAtgJC)COK^I;Wc~kSU0oCkzp#+>UM#lSJ3o|PT z#$_i`i(V4`(*h7Naz*|03j6O+@HG|pZk{N2A`;b$?tp&)wBj)g2LDOkDvUVZxJs$# z1Vt=ix?=jVv*GILp}|ioPl+2kr-_rlb##_ajuT|Sv@_XUUVM<$D8vd9X%3`Qjvy?U z=y)B<#4BbAp+Qb(Y9X1c6=aU-i)eJwr=w2$i2?fo);Qyvn?Gp@4ised0kVX)_j`0v zEuo4&f0J0>W(8(H*Udr`30I_H7hKv&Lm%-y97FUNMMvj>HtY9B9~z=0CIAWs3Jw+y z85Rx(=06Sb<2V2o3^pal7Zq5lA5NG!oL_?|sM)zxO%n6#df=R=w{XQJF0VcZ_o^io za4V~uZvT&@h$IH}uH>*(78KVO$P#$Ql2xJZ7vmeemX+HVLz@2&uvk%_#p4ua(OwHV z#b3UMSt}Ygiu9+Qw_q98bI|Ye%-JmG0o&gMjQW*m#<#|ghrWZyPVWqk^kBTJZQAZ? zgEyRbOUBPxTD3}bGbX|VvKt!DV2B?WZWx;liCkA5XcUVTxtFN7A;{n@E7|-;6(Z4` zjR&|!p8FT|k&w+_7EtF`;wU0@57Lf-6)4o7hcLXf#j@PA${T(g6~%jE7Ua%PbDYJ= z-0(;-%J+jDEEii0R7*vm%`@!=S#7-v7-#2;Cob)6ZG8$`=F=6+w-@zH7rB#kArvs@ zC*_e{v2=0Z7zcave0gIph8v)Y-APD*o20rxw(OE!vp`94|AP7E3ENUhluE9T8a~%#>5SfkZnp7P+h_j_I(c%qrW)Mk8NpY_W51%4rcs)V<&j zlaZ#1oPT2mQZ9aLCE!Fq%pKWO`3F!@Jh<97Vw z_3p@x+%QO81!(GS#Y{3;!N%4#yfDUFjYq{hm43dma$j`F6}@~aFH)FG{iba3j^fUqjdz$dlXR0ZDbto=T}SI**&^=iu~NY~6lMLc(D#^`Tb_4K zpM;bqu)n+FBLRl^{0^73S@5u3cT&tLD9x_81n=ZMX-Z*wFl<9d5kYVY!@pl{1uCFb zuzH^Vc2aPoJBefz*i{42Qwrnw2;icntr1ZMB|e~d6&Z}N$Ga-mD0=WMkC}_d0?HXL} zqQGJOqn*OaK80E=`fS?KgBJu{Oc%i&hc1T@%PlrM{|}(la9?PE!+vKs?N0dnT^Q#_ z)mZSEMyFeLDyx;*6%E=lkbpsE1lik3yjy3Q(t>l&a0TEo(=Zk@%q9?RNEbZ7<`n zGd{c+j|eQjgv_L4&LN)(hQH6OKWjbvB#tgy?-G{u*RS4h#CJUKIwwH20UTO?mqtGC z9t^z0z&be$Q!a6vo2w_3{*>R1d!oQgpM$T#pl(-IN9mw3`RofyTCm6SzzYe3_`UI> z5}+x55guE!UX?Yr;^Sy@+HdD3cA8sedy@igJFN?_&d?|Q1K`Ek^u+BkLGSyfy*V`h z^kO*tS!}fXLp7Bo6=*PoK)LiWz1>J{(GHa35B;u&@Pckj_R;HJuML6%wVR;^zC*3q zK4u4phqn<;S7dnA3lo?(Whx1z2t2|$vNr906%9eK2Mw3qq!^Iw=`>#OUQEsk&(sQY zU#jfqz&c9nPxKuLv0$rgR6XrtMzx>|mZy=ZpX-Zk*8uU?E`$#@OObNTQ8tvjx|ynK ztA1h7B9UUvBq+I7!O6X<%jm#fdcLWL1JO?yvWqsJ^Ud$g^U_U(opj<$^2HwW>Q6Ns z-X;DDPE-~Sa={3JiVn8JOk^UM zHPTPz|soxj7^wjU}pV{l`l!GSF@=e zq2hbJ(7oMIdRSX1-}>E8f-bwfDf2GjOh9P-uS2%5m>VfjVIrpmo^M%g_V20%c(x36 z>GE}mMywyG*Em$ardBm$-hHizudraavk4I?F+c}4Sjf;kT5L2}TTlL=e(T}SykcFd zgiL(7XDCT-iU-kVW&TS17!ehl-CoLH`5?KfL_fpJXH8j z9|5kb8OWzrdK#y;%jezxGUXq+xO^Yp?eIoRyCkyIQ@AXQf8JAXs2@=AksIRG-r$p$ zr<+~s3-XC)IA1=FjZ`<$ot{gA$^5#^Tisyk1Q%Oofx|=!4x{MPh+TJxF6JJ%WlBUB zo{tu!1+<|4@gI_%JF`Xd8RC(csLyCZUB}3ND$OZ0rGdbdLAECBx{iTaJW_844=AZ~ z4wV9vWTU2;+h4YvV?Hm6Qth5S+f(?d6Ik$|AQ<-LYeBWXuL*>$);R3Eix45XwC zXvprP$!3Dc-H3ErN)vW$&>lz__$1Lvf}e5o5)M2LLz83HLcVS|k$IQkJfkBYt9PH+ zm<7CsVyMVLI|;TL+^I{n)UL+_jf-Y(ATWXWCEL~hs57fHax;Ep&2VL2dW_&vs#0TL<_&WRO`#X79N5>UtBT-JJlB@t`H9XzGfm&BgZ-{c~135D!w>ZCU-Mi z5J?2F2fj)Gn`kzsi`*U(Mu0pd`at!V4eft`ep=2L;uUvZ&*Tinx`qTD_3b+sa&*u8 zLZfN-aoq=xFH@vG<=ATWFvh145Stp`-%~V$!1g&voG{#EVB(q4?R{{*o z9+mz;L1#s@tXxidcbe6Re0D=Y$PJTsVXdm$W|kABEj#Z*m%9_ zKWsk>%}*k8SnETma-hxLcm#^hV8SD)e7tp04E$99KI*ic>XuE-oN{%xIfZ0z3_w}6 zP_t89Yu&k?PUW9+SzC)iH@0WSmk~?a1eJd8r4w7S{|7*~e+Vyc2W5E~{q+SexW4|0 z;GLts%A2#8f8r;ZvoHevsvQwPH`g>Z)q>qTn#P(I5m+Ywq*yx&|wT=LM} zK>d25W-U{S6WyuipyX;DsXis3VKA(Zl~@3vcg}*pf6tZ1M;UF~dDo;_)ANE`LW9Nu ziY$DH4Ye})*l7K<1a%8_DtKgDzdsg<0|p_4LE&l6{cp;G^GKriI9Fdsij68g=-4K< zL+|2fJcuo~rh~gJXBfSx@{JRusqS+bEk(1&v_}DTAT@Hf7Y&kyG-P17pyez5bQ(k+=PN@N^h(&>7`6LnNqd?>y4J$fKh%a}yPRNT;LO zoxjZGNo`G9Hjb>vvj?H#MXs-L6-JQav46Usie5#XXf5p+_eYxYQ_ej*aM%bGNA?jR ztP`}Tt^;MGkB*ZNUcdr{QAV_|Yqhv}h4I#YoEUPWps0opZtk}SUyv&VYPo?}_`(h^ z)I0-%R7uhIy!)3xnGc~##v;f)*?O0)>|Sd6bstLE z3dDMTn5^Aw6KuD$Is)|}qe2Q+t0m(LTOg~4+~bGH%jZ&_6}4u@A@iz{e0Ck+1Pag2 zv}d)JPY6*5FvIbWhs>IX9$hBw=iUpU8FJ1(pKu#iY%S-wLt>HAYZQ0~)*Z7dDdMNB zxru70{W{>*k3Z9dVV!@e7}0A~G!Y`<=% zVe@E-W0m!4>!B!p(6pGwF{81oXxe-i859-*DUJStL(8RvcgHvD$!;y2TcbHI7KFu5 ztaFxMb_<5VL;{iwa3E5wL^#t=)L}bm(u}eNq=I1$d?}2S*@0TFo}lqUR8zAQX5}W^ zJ_=k&7;jq+{?UL!raR(h{5d5*r_0w7;B+b83XJQq2w~`xd*$rRn8pd?dH1w2t6u&> z*LAEifCN6souRq1h)1xhz^e)+CS{2)EZjx}UQQUNt)`Eju;Nv4!HOSEO@fz^!@3XQ zOJCB(1xEB5&nKBkU3sgHs4K&c;FGJ(_!l<&->LqQmoOUa@;F0M?)LP=4p<7gV{Qgj zGCc6brmUOXdUH1C>}WW}jg&R7Ue*km8>M6V^Ytw^el=((I2lgze#W;SZD=VfJGh5^n&v+*E#alioFNSyPa3aiT z`O|pDD=wn!%sT$c^zl?L$LSyPM6Se@4WcNX2)dv5Z=T^Cfnq3hBvjsPbRALknf-G7 z5wx*Jk|LEXO5Qg03y!rI9JBN1nw-;&;9*d+lOtG2s*`J=&6FfZl}X=tANY#FZLRm*xhXZk#SsN;FJ0-k%d#5uY?!FmQVPKp{J>$ z&ocUxez%KYX|m4arJ7T#q-*FHdo-oca8`e=9Ja=qIj^AS60%&4Z6|;E$-QX;>``ID z-utcAU;jI zFajK34-;Q+KgdISbKN>g|EST4BkHkoaiZnOdS#H8NG^a4fFRxZMP<*tMIb+I z+N9g6z4x-XxH;aoehZxhD|m*I1d)46IL)m)tbpAJO%=V&?eyWr2v4t~@>FzZ2BKHy z5c_B2d}=o3V2%#H{)hH9AYFbErIE0TQ+)jz%m*G{_C4${fJX*CgH++rqQUx4pZVbw z^r6YHi}y@lVp41=|DU5>Vwmr-ry`4@V1j{$sW`a@0Bmu6$?MQR0FozUl(LGz>M_go zp)@p)BjS`afSX#SjEUy8?XG2=tPe#$#I24dTg;m`r<5JHQn(!;$u-2Q`}piT>S3zM zP5)jOY`P}>_Kui|Q?hAY$<;+@>QgN}kUN8Pz${(KSx6=iD;pPB=Ecnx;oPbdQfx&M zK_1Tf+aI*Gm7rjmQp z@YM}$?BMS9%8o;_Ra^F9#`lex=&lU|1ZG|ohE+#86P9z!LnO;J^c2tghn6Pa3C0N9 z7pkSff?|(}XQ}|EKvDRPpDxSIvo=g3r?e6*@#Z>jb)C8w2j}$B!nk^sg#;UpRo!&4 zL^N0XqsCy9U(qTphb1EYG;U)uz5AM?Efziq)JLIcF8M;NSEdBC9i$- z&zk4Jorueyt_y@=ZD(${j%=o*YrS)GDs6KZcFarTL4L0uc)BHL>!6-vlxCjO>r&Bv zOjH7M2YAEzdUYOD=MdStE49|IvmH}{rDUTesjL?{>5B6o$6dauul$0}iHJydhszq{ zgPtyqNA*M2CuSNqOhW}LZ*M>8vWzV7(Q`>g4g@7@$5&}{sDqD<>l~0lH&9tbz9;^ zoehncO|~`pLh@<5!q?s+TFrYV+Kz|+mZ6El7$zA!id^ydb#HqS-_v1~JFh5!2l+R! zs%K5BwGEPeseklU%d-4zs;oOl2S>u`xTf8IvGndpWyFtYEFq7*{vkY;PhWe~C{{ov zUVmwp+^i>t&^=8EcU)N`(otugnoZoa-x{qr*#}j5eQv*8SZHQ{T1i7vKa}t|c5awu^-I&;LmGH2DIn(fb?xLA>mepBop9sa@x-5CXINC;=tzc! z<)Q}Ev+bf!ad+1V^y(P$HWwh0_7pfbXQbe7RDRm$UdjFo6>PXb<=G4KUBed|^K~0? ztY;rSuPSgV^mUo4?wlUY$fMr8#;M0S)$weUuKy5CWXN+c!?v-H7<)+d;V-$xHo0fj zow{Uvamb)-Hd+;jJAr9?O)m|9v)g^C-_Nt8Hso%)bhs+!fYg??n73!WJp>#yUnP;C|JQB13Yl>xJg8m<04dy0zNpXxUImjmQ7`?%kKbbbnz zF|};xw74yysoQp?Ce|8_%Q99AOBGG*=ed|WZ~OIHw)Em5(oZ2h zFd~w(?ei`<6j0l=I;n>I)Z~ukl17tB_;c59Q`iuROzw1RRe6IMyK)LE`xLz{@^{NW zRPVV2PuX`mwF94VzpD=;fh?8h@bQfQP0op*r&$^A&t%22uxw==KJ2vI0f#lN{MVKQZQBTQ=2QewES-SR0cqS z%W?I^p1e}GnHpE*H50)XpxBF3hL^qHi*z)cCbwv)FiV;U^Nz5S$KDKeCMixp@7Jw4T@J>DQLM^X9mA|6Cs zMe%UpbP3Ty`?9@nhH12;_4<8j)$GqirQUT}@Qg_rTM9}@dytp+Dy;f}>cNsf1xH67 zjN*|EarfbM%{>=~Ax;H~M*MAC`;kl_qZ%}4B;9Zi%h7k5mvbsqRl;C_Vrhz(K$N*J z($<_Ai1TZ%&R@;tGD7IPGhMT6_8L#hgZFzR!&Nd>Ac|YY;UR?4$#{XKrA~Llpc$o| z5*Vz5fzmySUs7BkO<_i#NNqXBKBUvvq#*U_;S}zBi%*o%&J|?Q5t=0p*mh?w9(5ud^tjA%bAlwZjDj+pD|mh@s-U4Vsl*7Z>Fpj!msUgtDU z7E0;K_cCeO4RpkR*FU(Xk~9-Qcg2r3a94WsDy5L{iKGBNvkhG>a~oNha*8ZO(weyH zBI0d~_nNlp^wAJkgNa1l{R(HBulUbzVdvieVM_xW-fiS1HO*CRmj10*6;j3u#w~C?yf9w-NW^=rCF%B$F4};KFl( zdHUQNmhV(cMOP9Uh>91k_P=XexF=G@Xp!XeO?zi7)x7<4_xu?N@1Oo4#eegw!P3l! z%9J&duU^s>Y_NJ-H4F;EjfdKDbqP11dO8z-jr<3|9+$y&3hlGpc4&Jq_TwW$xV~9I z!>N>xnE$8WYZ=dVg#wx-x7 z$hEBUT%aaXz@Ox0SPmlQ&sKS7@7>Vvph}49rA#2AxaMfHtrFpqTE31dtmAcQq5axA z(106HiAf?pOz{Q0g%%vQ`bPSUf9VsbM6anE@h@0yT4%pSZ%?=Pyl*VE+ECG*}`s$^M)J~v>gMS*T7^8Hd-@J{jVk_guy9!`A45}q+ zGhw=$jKHwbWo#s&ED*#wN&y_(vu z+~)_GTY+SoT&BvMPObnpL!{%Jrsc)zaypU7i_X@yOS{sHnlBD;v!Dlm&oH0~>Bnr}WSlapk^9CNs9h#Ci8T*1xFnWt9erx-}#l?A?_Y!eRJYn%0e z>QGCyWNV_+G%ag5K3Q^K;SmT>){bf6wzI|C8@tH3k+x-*awT2$5r#wGMg^_`_6mR8 zvDKA(qackAcS=S1xp!Q~N{p$Ue0deH{PI9R2Xdr?BW6_RFh3)GjMv0;g^`inCceWm zh|2B&aS{Q}^g-mgwWp*j{2I;hfAcVp)*qxp{day|h)EWG|1+@uoRTYewQX|NUf`)L zLIEANPy4>NzAP&F>*zV~@c~v?Q)w?X2b!@L&@0dn+vKs2zB%HYoMkeyM!<%LtFef3 zhbKG2I#LV6#Gr94g&e{>k9K?wH@Ze(o&W3lIZejZ=biA#z;OGWl}D7Dm7^vd3wWJ} zLxhaxzEwW5HV6w1__GU0wem>9JmDY}kc}$jl1t5M?jz?;Xh%Z2MD;qg zIijGgtG-&-vAQZOahaes`=_b?=@>VKZ?aoDak3YcMMeE&N-C!g)6ic$*Yr9-^$&om zWta}QS8p{VIJ34eq%6OT>$A36qaZ%ZSaRfS*U57&2?5a-D&%x@^=hFPe*2JwYEb^4 z02>(O=V^5+8V-iy(Y>Hh&#*K-Fky0#-Zw{P;ye^x2(a;A%tmv-W5^t4T=&87g@=p| z(prxp_J!+%y1P!PirZ#K)Lqt1kRG=6Eu6;qM(s)7XzJpOAcGV;m=1eb?H@G+p)hf7 z5>;|7uSg3f3Fd-WNz&?u$|nTG;k9D{>o?g(P~>M} zV(ym}aM%?&W|hJ20nm-0z;pZ=YvvvK@gdtpaZG%{+C#A119{G49L_QDxq*MI6^Ok2 z5w`){tf=Z58_Rh7#>LIL^MXD=VDEVzCakIS75ur3K|sqgAFw-R3>H8=kP414kvPZ)-2|ZT|oVHS0FL6F>k8 z#nC!tON0kuOP`2HZ1~PYJ|Syqdn{MdsXP&^VtsC-7JPL(5?*Hu_U026FlVMZM`w}- zx`9JDu+hM#qtxYtJQI2Jw4_eRbsleEyp&b;N{H2)Pum?qnzXjVw6A=ytT5-3 z)FsEiq?eCwbsQd8s#$fz4s-_I;(XHt!jyKy7~mS9j<|W5E~)p0xZwe%v@2!7ZQz$p z4-8OUuB-UzAO*O^w5zp19EE1qIcg$}P`O<%>kwbYC$1iKar3HVG^>Hv&1Jg0!Our7- zF-EsKCe&tZKcC{R`%f*VrH32N>Hh$!*TI*`?b`D=UMA`?gKm~aNntv4E(n^QKuuy$ zeCmkC-9Fb4{I@KA6#TRO;#AZgQQlL-+|L@uAn9ruu3I5q-r)3FvsVkg^hGO@ZY6Yn z>Hu6Ff&9i(dscuqPjOk{WsYk4Ohd3acE{|SQ40zR&(u_L>2u6q;ut6q_L7=T2&gZ$vN6M0AJIqjD|+O2#Y z97^GBS~ZR!j52;4N0td`(Jfwqf?AvX^7mgaiAI2#)fQ)dciC_25kWMDNTkt=HDzzy zE09?fgsEQfZpy@|krOfL`dR)ZpqjD>9z#V!7~q zd4SQrL79=snH^5Lia)N*vqKOWekS)8%&$_Ms6WbT2-YBVSPYAk)%2-}* zQ^*cG9n`U?x0-I3gu*cO>R^y&-bATo^vTPoz{O}dDk!#PQoNYnW<1t8qcPY&_ZuuL zpNN?e2N_%+qFnCL+YFwM+!vd_Fu$3l8(mS+70C#`fdW%!CkEew7-PO(!`lb$XlkmJ z=otHzZ*X7%S6;Ie+_y}C&yDc{nGl{sFZPNoQMK&3gg5Nd3{`Q)Fa72I_byMb9}@># z!hak~82<1@sa@7sV{KNimKwlU+_+_9jXoRlN6P@MYp6W)4PF-U$A1E!BniN;ooC?LDOr#@I*n#2YtuZkKF|B_@paz{ zIy;=DJo&em%qKOV^co%xW>YV$3qTyJwqMZf@T3XLI`Bn>Xs1DJpg2HX4F&OLpt7$+ z$~>Z?Wu$Zdf*etZ(BL_bM391nHnww76d2>AtUzYBl)@_=G0>Igx!6dIIhtL>Ki$UI zp*z=7^^7?sF5ofR#$8KKT|woM^I1b1y)mhN7h~a@jgRGD$v~j146baWmTPjZ-4elD zT#)fpTDW&#C6n4kt{p#^OR>D(>*+{??TzI>I$Q4aczv{ft_ zzo|t+J+WF`cPO9124-*7u5nfoK7jHE?Ik@3gWC@2%|r}VTW0&OPEV-i5L2UK+d3GHu+c9FSorUOE`@dUEt z?&)uE3d*V&%{%ta)DgAWa@O_%^cKJ&wAU0FmTtlZ`X4N_3)MI|tKdK`nl89C3mlZ% zx2!MSOotoR6`jRUQL_7r3N&f-*XlCys74s4jeSCIij}I{a4Wnc+`Z?R0s4ejgK^D} z%aBxV$5Dfg-{n=qA!{k7mirlJ(6fLy5SwAbMVT3Bv2!AI^Qo@6Z$SNZDU!YvnTUCu z_@!!{(G@%Q5>> zf__V@lp4mihSa1OrH=ZQ;|BUJ=0s?6-rA`7gM2`Xe{e#p2UvnR{2KwF4*^Jq0n-Z+ zJ44KE${M5{!&6hf6D|Fdh}5YUGV0EC31e#0d`T+5l>AnE$e;` zM0q6Qge&Uz7+$LkjUv}$1Q!8kV3a}xNXO>YW)$JvEn7Q)k4&(u$9t?=&@ z_YO(3Q+|_ZmeP%yFUH6sBkWe)!a%MQzV_j3z{Z3B!~h}@00II60|5X70RaF200000 z0RjL65D^3tFcTCYKm}0$+5iXv0s#R60RA$5p#K2DCHV)CZu%)p7RKa$K;$X@yDobP z`YY-wh7%F%xB3aQkoFf!et$hh!G|5u>bLocw(A{HL9_D*DiJpdn)h9b^x6E6V_5E^ zebGoYt_%0>j-!wzFI9@)#ci7=2<0Q*c^{Mc2J0^See1rfxj@7!c(#8Hj9xi0%_(if z{+gOmU5e)w#3&bXZ z!wfM^v0}uCHLVzBfX9jYtf~DiCkqT&X-Q;Ip#sG?FvlGnD2o)uoI~efa!xwcIR>Cp zs>L0V=KGjoLYjzcIO4ybA>zY_DK98Gd<_I`PTAG_iK7T{6ymbJe7gzBJ6LeD>Gn8N zWD|c*`?3=h?M~@FEJ+wxO3Mvt*1j*u(`6X0SrjZzD)C>7Vi=fGF~vSSmosAyQfSJq z9IIuPS!03_5~j0ZHhpg7n6bi~a8Hjg?Lj+H61!oFQpLYyn33YeBG0*U2#FmOso))ELaTv}0q`W4VJTa=HNp{kk<@_>OhBPVh;d3PP1x#IX)G}2 zr`d)icwuEc8vHIooopr~uudzSnXMF8`We<~KSD_|!NI9Vi&C`mxbne@+{HVG8d~~v z>4Ff$33DEBc>aZX6m-EedswKbwe;!pG@4B$MLV#5j}Yp7_!N&Qy&R7g)TOC zTOQ(Ez7cJBO=F2~rPfEo&b@WA6AD(!ul$$g8w-+5u-OtgrYWY+K123~Ye<*i#fy#P zlMWVJ45jRpLn*mWi{f+Nt5Yi@|x!KQZJS=ifHx?+d#M7K=5v^-bVz{TYa7wzGV*cB{&Y?L5 zET;}uImp3;YgrKqi-#5jGJqTOCfhV|YvYO@Do4hm;Qq@JcEioyf-K_B>&QzmN6do^ zeDOt))?pYTDSy<(iG;^p5MoX+hAB(Y{{a8Q04osy0s;a70tNvC1poj5000025&#e& zA}|9&0TUxKKvEGQF<}HjP-1ca+5iXv0s#X*0QiF6;RJtZP`=i%Rj1a1H_Q=)R4_|` zh6#L?jqQRkx{#qyv?2EJePesRI>w6|-4+c009nx~(=3V=`GfI9VG(T>cSVFzV8O+x zN-vG6H2nNOEMrp>SC~sYv7v(mtf5$|4J~&3B^X3ZRK*lxjP9Z?CcR>lM)3Un9hU|v zv|gV+wcGF%V-+;kwwm%OVx-ZB^UvFcwY1j(p@RkusYb`QnMI=$W31cYZEAeoxobA~ zo?tgC7CTQ)w&&D_F!Sp+$7Y(XZO^pry}1-|)@wF`BL)pG0KphBN-WycxfqunmYACY z7$t_>8qHS*4-O?6-HT0Dn5WuwMs)tMeYdRInlSAK6d}*k@ZwQe+#Xpp^F7nd*kKeB z8(SE|tRfW{qXdOlKTqJOwhIOcUbaw1nzp8jSMK6twY8p`at>-yhW6o6j8UM3B8?YW ztZ(V^T+0?HRMqMi3iOIJS{mAGJrcx5H8D+f!?d^-y~V@nMY0;&v?1Xkjli>9Jiea` zbjV`hrCQCDrr8wg4H00t>q=0%$FKH_;l;ESSY17b_QgtV@O8E+6;eqv^QJq4|w+&ReRC7^_lEICOLFtUv zFxr%1=4>|SQJS>Ynwlv_@ZwRMX2orRLeE-Wr72f*Vd7$3nkckwrqsUJ+zS}1Sfv|b zj5NbaYr>;z8xGu8+XSqvZ)n3itIMS-^@oRS`=e6pi4VF7L4(@}t!A{*aod4Kqf(8r zEK$LWl}6iGw8L9wv&<@Jw#8d>?SjS|+Pn)DjYC^jwXAM^vSFKGP)f_GLwi=UT3?N| zYk?2*1iiEW!~i1^00II60s;d80RaF2000000RjL65D^3t6EFo7Kp+)R|Jncu0RsU6 z0s#D;Gyeb$X~-_E+8LVU3H^-U5XXO{C(2NhiN` zv$=J3DqLC~HVT#ZQk_=t(iAK)aJb^b6gjY&y{TA+B)lA$W3RHIl)_?xHsr?w5Rpncl`VsfPC)gbK)GBX zSzZY=K+FO=%Bci88q|Wobgy7=DKDrYib;TgVG^nLx zDoweC<#O?&6cH>Qm0S}LMKMjBzdGGb>lRp0!Yt+ey17X9jG3#)rhqb`1I3kxtnn6v zqm9K8BaKT!*0fi~tBI!@A{ojanlNI-1no@$M?+bnC`j>18hZY^*A`hGDKO4Z^wKb+ zxr-{5!#53UNr()1u-2%iEHcPo$6i?U2&hgAqI6i9S-$SI;Kd#Llig#>*$+`Q%k-{gSHDzR?n5AA6s%cfpu-k% zujTw&{BO|z04k4-zWfPe)`k#G$?QO(VZkX0NwXBGOYK8iQ|EIR3Q#zPSd2Y&r&vl& zf+ayl6zh99IkD>y!dn*BeM%B_I3)ohRuDMCo$qH036;YMc;z7%r>2y{#X=NCh_!+p z6jdB;VTuqsL?$`zVilZ(^i+!!6V%0r6oC+!L(-TjEv_9CcAx zNTi*sn)+!%rYw&&JG|<#MV^FJ3JhEviCCv69s73P@ZrXYbV;H#eHD(y(jc=dIwV1d zo7$PY*&HP5vdYppj|l!Wi3BkMM>wVkgC-cc)KMA~)>LT6gc?nFS+zTIm@{TpQYnP1 ztZ<*o!gght-uAHPL_H-x|HJ?+5dZ-M0s#UA1OfvA0000000ICI5+Va2FhD{96C*NG z1Q8)(VR17-P;&p;00;pC0Ruk(=?nfLAwu#+_j2oGgm~nm)(R9Tc%#$FM?zn+BwGp+ zc$Z5gBcxQdla{g`l1e&75ok>mC{qVgU{j*`-@B%=0fX(U_Q$BIR*bcnBt$V(^5N#a@v zltq51P?C`8CchP>!{${8iY#?PD>gim{{T-SaxZV?WD>Bbl$46dv6V@!t+~Wn+84zV zA=pt8{Zv$1%`RAu{-af-iCci4EmaqbljvK~`w&B%;Km(I=58=_BEZD(N9>D$83itYpF@g{-7HNS>&1ikB5p zNqL&cRbD!m4VpxvE6mBvIR!+tuHsAfL-JA5uOd-aQ5keLR%n;@(Q4IZ!cjyx?7cY% zp~;MhzaL~pllWfhti&-{3ml1A;wq}GQ66b1wUwSiq%CNY$%LdokytYiyYu3+V~>|* z^*;|4^#oQQw}QuvWJmW|GJoX=`eO1@C044Db4N<=Sdzh--rpbLeNI2|<@_(JFLqTT zwpH}ldu9EvEMt#Ij8&!7Nvh;3NK3hX%00>|Pa7{Vm6e{o4|7C#3wc*at8&RlS!*6k zYb8|@;*OPQi&^BN*4j!U)>2VvD3e!)Mam?ITwIi1B+?ZUe%^}8%36tgQ6fZ#TQLff rtIL8~#C4B7M_88>UPY{lxdA>l01^ObD5(GJpMZgZhJizZgM)>ILq$Y{ zM?y!%z(7YuN5jM>#KpwI$3jELCBwrfA|fFn!N4W|LPq?BkeGz{KNo?3fq{X8g+qaZ zLm|dQ$0Yv$#>W5v10F&QLJSH50|1Eu0fhnaF$5s_yiXX2PXPWGFpyBtuy7Cnc!bX| z5d;7d>i>-b0FV$+&@ixYAL{^QsLv=m6#D0x6BjcLRD+2sFA;DBt}>cAvPt#4BvCIb zvZ|t3@}X8|Y zQ;Un3V=6{r$aw+&fL3MjapgVkS|I^pAA{|X#4pdJrKB`5J{dC|%X3oHZ@a3*)CIeA z2~0(j-*!vpEYc$|@M&Z~F^NoVFYvzbbNtx9++=PwMO7^fg3f_mY&OyT)Q?xpfqp}47ThWMI3QpWN4v8d znn&if(lf7#C0&%f6uK#dLNFjqYzQn%Q*h8H(uEP3hyD`drVZ;K1zwC#%{*`nvR#aE znw1SkFHR!*$M^4>h3$1uj+1KL^9$7crtb!=k;v69XBex;*Dxe+HRsUEXs5axO(IQUMnyNZ+IVd$E-S}zr zUF!8QP`PWA4$*g7&OyrQf_ZM0j( ?aMLA1vXlt zVo?#{UmxO?2lcEuaf_CmLD?sly3vp#eb76o#zSZz9cFO z4wjru#z+I`l?aH;ivfdMca$}XPg%dK?|s>2RjBxsL%%#biI4M$5?NH3y)ZRff~rr- zg)XTrW(4g0ta)1PLdhKkdFq>*oQX2r7&{IdZG%f(^TT^7ZMYbXL*9iVO4qEtNBAe-kn6(~x?0 z)cV5Q@@L7s+ThhWpT{@$Mn(S?OxQ8NRO!~4u4v5ebfm_0uyEJ0R^H$vB;O>A8aCMA z2Nh5id(=7_S@$ka(mf`;A<=0bDKwkaLshE}IYy0ZdzFAE~?S$St2Rc3m( z6`90jwz?*C!ZN38r+8^uqscOAO`{3q%68iFP}jwqUC#Opmlap1cho%ScQ52U@P)v2 zAarQaXoi?jat&w3QJOVllpN*F3JRxGfcn!11xUkBQri4cnoSI%D~Jdm4VPPM9t{@> zba4|FrK-;MkI98!d+-meBLTt$=T)@@<|3u{9Xi7gCs*l5=&Lg36EZlhnRggj`miB= zIaQ-k3J0xcu^Up3-|k`kd)-MVsN=Z(T;uMXeLeN7I|qPti|1Hat64UGY6`|(bz0M- z9~@xPT@9ieI+GvFhaUHS)@M7`@r{Z@wn~ssqA-9COg;bz4w-Qu0NgD W-+LJeh z#NS%d1H80re}cltFH>!nG527jx$VQV)}>li{V<1r(X6*K9O+ zZ8<0L<Frs027ImLFuXNM_BYa}p-4urr*IgA zC+!8^7Pk`_srv|T&-+ZzO$>ub5_!6$^tlMR)3m5@>skmKW1NO{7D768i!7G%oWNU{ zYsvX*zRAsW{!1+L80Lm3KcOw;W->u{CBt99R_l$6)CR{s%=3-~_wL`qmGVhEG zDXRKZkG$At650L;*2o2t#n%nUQ^9y!we@mVi7Vj_aqFvk?gvfrD8Q|X;7+Vqi|{Yj z&?L<(YFm0nWmb%mm``<}45Nr!SSG6|m9b0M>0C5`U3o_kwyWwv40Ap@{y~cOg2z4D zt!pGi4EmNuGt~nc}d8bd|W2v zWr*cvO+(?498P4)k8c+989A&z{jQP@cpXOmOEr=`d>xT47d&=cn-vSr^~I*~HL}2E zX!GfZyQT}B911EtPQ3}wyuc0bJX3)Qkysc1Q{r>i6evEkqX|oR_pf6roKvDJyBK?#Z0?EucoN`7aaZAbG8B&7@+f$$Y0j zL#`_Xj?LhwCCHU5W?{>hcEz;jmdq{=L!fO0f0D3)Uej9e;fBSK>RVizHv?h6TzwJs z;sqBBh%}B#W80|9Bi1u5&ZIwWD;lPGrM5V1{WKR&gll{5to3ffkvDH~q+DpV`D}PP zeS>i72~h?|1J%ji*_8HsBohq0u)I!@gHDdo$na~+N*)SLJW|&3OI!IQoG1HQbZI&v z*Up^-2~`ZoUw&wP$*)bRHtQT~#$7mcB+(+86IZV84gMToFe~O%&*fD{jTctQ5Ma z+Vt>y<9lARMj~y2!g8R}sl3*!HvQ=%ZAWG96C4+#H?BUU8e$UWLY+^b2kFmYyE@|N zPPcDrcV1b21@)Zwo=BZcvX3M3i-M%Vi`rG4wN3$dT0&=Drnu_ILV{Jd%g#D2=kL9R z$9-N)1GR>UFgxiaikfwde)fUzZT(hX#!Zr%egHD6&t_xJ;5xA=1Q0whuCV7xp9>z! z;y0oKN%a)J@8m+E-WqkHI-<+g_jp#Plv{qO`7V|sJSzN{^U9>Lc>vt0x(%k^uQsUE zIjD^rA5lEs)&3T3Wu(OgT}yRL$;6S|ukI_J zD<+S;$kD{|#gvAvy9VvYo5Rl{sxdMhI0YHa_TARWu(u3>)1!8(nwLm?cFx39?RrxWRKKPPn?l#40m@Q-K$^bk8Vv@!lHd->lo_FV5 zwSh9IJ+9m*Z15dtWLNp-n9X^bjcBb;<7ry&xpIaQL+|*~H0_7c%`)_00fB9O=5ru! zWA+7S5qRYm6nlDWHhweg>w1o2Q?tlvz=^2RBuQ%r?tB)9Twzd``(3Mz*@rm&ot!>> zwS-nYxY2pNrtGwwxzSNC`z)x&06|T3%1B~a>FCr3oqf?pK`hll^{pvT=Cpfi^XW%# zVo0M3lu)*WRI$QZ5h_>lA9}xKiOmbsscMW}h5nhvM2GcM54x%SBlN z)Jq_HvoY}H-&2)GvkY*jGqYq4)%6-c6Pw)YHMnm;b6A+`Jp1;cHxgcj9dh69dFWfvw1`jepPf>b~>M%w%S)< z^pyINtc*rR+S}WQhT4ae;Kq?QV0?`9+E6dG*1Iamm`TOCfB4mB&W`1Jj0f8{w@Sf! zXDT4@CGf?|YlBr8e>~d~4lg~=Z|Y5prhNs7{^Gi3imzzkp3Z$k!B-dVh#8LvrPH_Ft!QpPZaoK0hY{I1#wb z*14=OI`%8;0UqaIs9tOM$(A@62?*F7a$232)@U(CIbgmdjdhl&%B#UGodfP|hV1lO zV$KCcP@3dyzSU0#=;d+89;HeZeOc;uY)p}cirLC{-%RhRw_be=Of-|F1T0QRu}8o} z;#u(Y8%r@D?qKMYsL{(=94f}!Y#?+2)<>B?0A$9u3jjKnlJiI#AaSt2G&vU@+|tFm zCQNMBiT0Rtmqc=wC7upLTPnzqWbUm4}iqo2Y~l3nDw@guDupoI52O7 z`N&~&qe|-=3K`7?uGYKgD5BqVJ8yzBFX}+!ka;v#!Z@!=pv)b0O@6s$pfB3@WyO*R z!cWO#^vR8>YS~)FY~n5c7Q4xAOT^dIRn7=jC?s-ERE~#stuv0+AYCIqdVQmLY^gn3 zqsktZBtnr=th0rJbmZBd!|lrRLFHNwa| znRnR6{H8YBcrK4LP3>Ay6FE$(b|dc4ZU;;}vU6SSp~5 zHG5bBs(hE0<`x#doF)6nE>pACk#Ll@c)@33t87M`x#U>-Ft4QT1sg@w4?=3(b~B1S zusH=PajSCgY;A2&-M&3*W0E!miAszVzAvvqxc%Wvi|}%D8=|0BiA}v}MgJm<^U?ztN&Db<-h&Zv*W(`^tCgyvZsP=_j-N5uyNSuXNdh;Tv9xHoyi7H)}`gVLzBUk zL(0?P0Pz4>HsR-&n7i*0RbPA%IhLw53n}2fu=I+{hNDOP0pHR)eC=qsf#>j@!#(HG$UpQ*#hfG_uw@kMv9nampsG7lJ&C|(vh<(e2ciF&{R5VHP z_CXC{^;QfjlT_tbi`Uk9)Uy%Du1_n1ja6H$a%Dbp@Hf@1_q=Z|wF+`7zcr^`lDx$^DGaFnyKwi?8AyzrzKxi9xh`}IJ72|tg;a()*s`4kF&b&9OXe4|I%&C?Xd0Ga%;yn ztq4gAa%k^y*;vJ>Fu{?acgIT7bGMA>$GmM?pDx-!Tl^cdnc?2F)-dqc$lXHYnlc1_ z`FW_LW#Oq-*dUcYpzQKZ9ts^l2DB>F&jz@yk!IG>8hALZeTX8E@kjk9%L=r@SnCe# zkFht(fF-2a*4=UdLYopw$P2Z`-7o&i^_?eEvwc}4w)0ad4KmYL&Z`Xp=L7OLZo7NBV8ubxgr+D>CU$;{;l!>&?Z78 zk@oPKUr3A-Yi+cYCIT-gEVVDM7}nS;Htp~4&y9$Ve*k{s5w`+8C=WD(W^oSA6I;vH zXd@P_UT2QrmRESlfn$f-TF0-=%S(EY;jK2>mHe294y~IpD1RSCOGftjvE2mR(3bU0 z9n-wA*Mnk6*GpuaQ@NpAE)9mf~yO^6oc4G)8o2xNtJ0p^2loV&2NUB@0L<@lCN;)(0 zBd~ZJZt1TDVOHh4X?Vk@g!J{6JvpK|#x~($oUx!(i$yOiXeNtQ*)#6vJzu1Oq48C# z=)4_chl1+x0m~R%5(^DHr>V4pYv@6nzj^;*ghO7AfGJUM9v}ocS3S0GMJfrlO9~>l%K+aYP zQ)JnW(emiHJ>22BIx{4@9g%I(SV`qbg;lr`-clof(|Ir*IhLkVb%OMmm+MWYe{Fc$ z8bDT2p0x{`ys*r*g4pm2r5vA+gR&-sDFDIppR`2=9RL9Z2?YZU2Ll5O1M}Z*`KRxU zfl0xR^+jCO6o$h!6djxLt6FlQS!4g)H7pLN`W`tIms{8kn}nqK0Ir35I4I?Rs11}T zL|-vl(fBYwW{eb>)QYmy9BCZ3g!*JHkjnamG$f1!ojBs4fB`Wv%zK8Fh6H?QLUfk|ZO@!EEV|Q~(XyI?#lT=k zcJJ6}QQ+!0bW^k8&1a>GE$Pa6!sQTo7+xY#z9yZ#odcW`v}`6>SkAa0tjDeCVyjBw zEMfZE1Jy`opz+sDjGZ6}sHPAfMDmY<$GhPJ9^ zELeiBh9BFlB4)o6^Ty)4C>>dd1iUin(Eq_WCunVCO4_%74)=Iof6hHEw#|-nW%Jo2 zp5uBZk-@I9cUUv`BL7_+rQHs}1t&emMITx)8W`TQ0Uq4XL!Nb?%-bL$K|}i2@#Xo4 zWyCHXD+8_nb3&_4r$f(t06eIzfIJKWHUFBa_N1+t=r05y(=f6QzgvCCZRDt9*@Y5r za6h5{JTQMaq2b{Jr;*XO4suYnpH_y$p@5SpC=Nav^xGaI!PwRY5p_y zoEZ$W_Os!fp(=9JYd6WatW!P<=y;WAb@o7r29!iY~wXvACTb3CtQO zt`r)XSzE_B{~E{wT!?LzIo~ND$bNRc(cmU(uBnI|{ARG{oc23~=Pkf#LFi{^Ic<6B z;U`qJ{!OQ)%l-SoFrfym3gvC*N5cuN#*6E>!&7hyUqb66Yc9k=0NM6!K}9|1W#6>P zU@kbHJGAWJwTOXzlZcP33Uh=kYZ1NPK?bR3jgOvkyT7l0`fW*-B7%RD6Uk%8w^)6; zVw?imr~hdOCx69b|L=F z56Qm6@s%b;x+4Vh+$6zoVwWknx*?6lmT6*T0MkbmAcxXUu$fh|!uBpv8ed-S8Wbj3YA+UP*y&nx}i zq0!c6D8Dah#lPF!AG@2R+LBMP&Ky}wyj3QO2$dC#egcfu)vru7HFYY06tPp_Tc3q- zv|$~UPFYkJz$KW4>@w%GGTS>BE}l%o5dbJXR>}GSU{blYYn`wFmLZnH%74^rtV7)Y zs&*bw+Q!8ZC=%EE5?O&*vM=tZ4JM8=S5nRRojcPQ^!8n#ll=yx11z5i-E7ho=?fBn zU!O|b8{)Ti1A(CSdW~#s{k9cgdQmz(pqJ*fjIzC4WLG^8Ldlmtdbf-?wFmoqY%te9 z+s%bIr;SKze!kGpMW<^OUS#XahrheepYW|C3L^Ycq2W_qB;#OoF(I3_QR$50?gS|o zUV~^;j{I3udLBd&8gBsQF18;4&Y#;7@=j<-w4yf4;hC@#Um@1c>SOa(SJ^o;aD*HL zOF~L4!$+P(`|-Cjywlptd1eL!1bkdck>F7sr-II+YL7Ko&u<`j(!T^OVEGj;p+!!f zcLavTJRy4jQtyS#2^{(WMDj>qE4uL0N`A8opqbIu)`oAWw_5hETf%j+I?C1Se=f`S zF;1GcC!mGu{SFi`vtKJ{e8&Pc(7d{@9h0xv}QO#Qn#DBErF;L5Vl zlwoV>J(oh_R6}e-Y-4oQ0C7p--6Ak{z(69t#D)02P(LVy@>}Rh%l|WX`3$}+rq{L*T1Q6a87#1<4->q>M^MtE$JES zZXq02Ce{}x+|U$z3!%+A4gyB&A`rk;e0P&Zh&aB3Ij;4G?a7=!yKCkv6D{MP`Y?yz z!ikv|_};pYn(c*4pQRk_+|Xpd4y53(Bx|KvS9^>lH=*K~wyBOk8l?~A2D{ev%IL41 zd#Dh`v&o12N)b!ru8=^iwXHP`HEl{W$#h=3GHy6LtXskOTpLJKico1>y3$OMXM3^L z2mzbj@i?*ziw%d(UpNgh3^mM~rKl)+nf1!RCD(9|x#-@B%dm29!=HF4owbqd9cLr zs@>_Y<6ilGQG%X^>w8@;S4HBlL)@ekSkUp{E%hwHmzK52D2Gm64X;|_FRD?CoJjx%u02tJ4SC>T9n2xHhgY)5E8Z&SPn4k;hn z-d^u+)0rF1Ql(+$oZoV)${HhbWFi)Pm!9A>#X#>-WuEx8D5HbY&pLnHQyXzJ69yuUEhD z&Gb*P!YcM`T3%@z=e}O-*gBU$ICDNN54!$QmH!7v{T);loUSSUgjn&6BlD|D1V?p@ z{`HH|XOiG6*)0gp`PasakfSsL)GaH>QARB7i6{~!BKx@3>$T%B(9%kx_ifsVc@u>e zdtx8e9v!Kt1};*>f4qN(*)VK@v)|vMxG#m@8ZO~AfVEG+3ilSsb$&6Tz&A#YYfvL1&vO0BO z?{2HXsrvZ6Ze&H7czSIoqzy*#GHOkHjE)}$es>8?3UF9(TzLGs`YBT&k71TfWDus! ze6{WoSXR8{Jlt9e51yl*dp`gWPsQ5$D2({R)is0{j#H6uryci;-YDhK>ECcq2KuwR zXTp<|&h1ctXDo+ov?Q4)Qik`K^f)&b`#%xrvQrdOMpmKD#3zXjJZ;bTEc z{J8Tkhu7N2Q|Vn$c@8_pQ|YX$t|DHhQXdh=V^W+G;6;b@2+G&4rSAOJE2Qxws!H|U z*AZ}(=Ojuk45M)e-h=5bL=pd=%8iMU%#n4rJ?9lJ3g3PWW3t;gk82?zjA%@^ zLH~f{GA}rz^Z3lZX$$IzA5Cgf+y=_Mzc;ZKap>b{qjQIsyJ{c(9#j;uC|F%%>PPdW z|1o4Xb9EZv_55hVJF+Ay@l>xi#6A)(_Q@ZXU2S!SSm~JwRsZdkrmu@MET!Nk-%RKo ztZNH=AX_)qnO#%&XD{zCeNdhA66Vqgb#d|ne`o8ATneh`tiEA1)Q-I|qQeB@@-h;Y z?J|;_Ml8P(X2cg`HhF@iVLd!n?`B<`yEL% ziQF7D9ACm9$QUWOZx_hJ6U?>|ie<(ZxMs-nu{=fuT?^1)!g4R4{NpE&?%$S&vme(* z1GnjLnIqk;pDaHB_o8fBg6v3q!~^l$yJByp+zEo;XSq=BZ{Z23Dn0;BAj;X>uMBxJ z6@xBdTrx^e=wjW?R(lo$}RHBLH?ixy9I4F@)fIzCt7iaBx_E|d;+nDB~!h9GJ~HM2Jl2}+(tH$BDs`_W+uuMeEa z*W;u1n?v=O@kD*bEVhvuYuc2(0x(|?fvxUD-v&5oj)$OxaV0kycTl_GV3dwb zU6_RWRg+xCJ^(<#2LR(}+SGPdlI^dG_xYCC5O190-PfOwL0OUCvFG^zbYFSQl|{Qt z0EvJ9p92XIa;=6(5+;5UU9urUaiv;A@-Bd4;CNwPKF=v@sC;FR(Hy^&$uo40q~_WY z!b!>bi-5Zlo-LKHi}=h6W~YtQ6ZFX;lj*5yiO0En6o(TU8*p=S9W53!NEY_Ul9<-_ z0`IgSLCEp|O|XJNGDYBk2;_O!=}&5s`orO1%3VLB9m2+^c59;<+HP&l<3MJ~icfDm z3|6?6h$i`76SZSN>LpEeT!EinBNsat>y`Fi35?b8is0u;|BJGjVKZHD%^Y?QzR;U3 zQl2Ji*Sz`13;i1pr_c|Ih1?ESompsX9L<;uc>bUTs?Z4O_iH+@vdy6p2*os$}{spd@+-}5LzdY!V zYzS|jd}tALe*H3A+H$}tjSaBQNJ0C-U(QkQpOW2`d7Q1eA)F1ZKRep#w;^@f*pkq+ z>COy_!x-b1(wZBVBDru*v4>EFGX{f~+M%G-?H3#HhN4MgnHuh++ z%9_y&Q8umiV|W%;!T?cRqEB&{@SCX5Dh}7JF{c-}x}2(|fd4y_fim|x>IWbna9nK+ zKdP&Q9Q1HWT-t@4_Vs|%G=*(vy?KMf=U#1I5GSU|2RVJ7B&{=ii4f=x{e6A(juMES zKXkdiTdpYQrC&P)0hJ4)Tk?9KzI1_nHvlmJLmQVDH>?#d-s_`X5P{#eZLdPW5Q@cK zSeC@`p!oA8JXxi0U4obq^#bHDAPFEeZed&)h_W+BGbeI@z*J`dRXV1yoX3>}TdfUS z*?ePx8bzIU8|~NjvB2{`+3$u=2kwvA3n?y%y-GqcHy}@5^TqQ`*Oqdm=&yYG4%_F6 zQWA3P)AYXk+wjvp$_UKy?NOw9##I**TKSwRx`@}6 zPC3?X^ab12nNd~1zGX+kLq=1D~BFZU|( z_05ixP16#}Fji-WKK6NpbK8?ZkHve^V(Am)hW2iu-Bm%pYBR!v^-_>gvwO@xxgbb%T|RLVl~5r-Li_n2S|$v{k=uDy@X<0>?|ba=+6e z>+K94XUl75H8Sbg$Oui0qGzv@k^Vg>qCCiiXWr4K+gt%jt_+64i|$jYB- zDHtYK-VZ>-&NetEK=O+3t=H!KT3?^{;dQsw9F6M5?y;jbo~U;*?g zaEfLaCdO4d4Nps-=AvaRh~BRUI*r{)5a=YC6R|b=xDXmsfg9^-{ZY2u_bk17*1X*i zuk5${+oT-##c-F_I*!W8ecy$o&cEf){>1~4aRiJ#B zSiHVKKKe+dzbKi5bfPTbAY=9$%F+u$5qDNQw&=Dm^?8_(=@tcJK@aVx1eton(^YES?a%ADDe_71qcnq-*;N;3MqwjVo^4_@&z{kgw3I z0pUjhH)%!)I+5$gEPsB@D=Xl2*nhtW?JK~_MF?mzNVg0kh3Cp2Rgk>h+i#frNf~Z2 zw5uPG4Q>U2d^tShH^G08qfDuejnKYCJ2nHZC9m90Y!h61;5Tn%f(|d+A)fUh7EcDL2iEP#bAsIrPeYrK05Rv1}Q8>Rz64{OLqw+A+O}h&jL{-epL6{r1n0De7GDl4DgSFB*T`y{%5{PFqO* zO{=g$=AC`pd+@J1oxcb$$6I1kGkyLKTOS(&oe0lv149`;>=gx3BMKHI+NbedG+9GZDI0tV(|gq+z1KakaqQ^vEbby` zrML4(De9v2i3}9tO0;fuX}1~bl#50MSnQkuM|hpF;h$AG>01g~v^5pij!k*o)JKdG+R^S(=MqN}}nV;xL-?%CsY1y!>lGXs| zBO!LUy5ts&uwKgRf&d<6|4#-6&)q7XyXa-7Qy+CIoy}irh&b+2#=)K-J;ay6K&D@L zVM<$rNVC?4OC<3EllA`Fg|AtQpR3(v?@l3|J&3ayKNt@tixbbR|h zH*3gwt!29smST|EpQqg_+cA|A&>lTYTA|+Z!`|UHb4rs#C0($!8BR_7jfRGBx<9-1 zE7Bun^KE#uc~RBRPMy23e=wSlXf`4#wZ-U{9i4pc2n$?MRhf!SAb-OwUbUWf^~fb) z!h(jZ67xPtvb(IECcEO`FbmVTEZV(&^}FaI!0R~|;vnsx*&pR8)h;09qRWhcdcT@UQua{bIXqVo!?7MTF*Z9Dl%Z6EXj>Dure6LS>KY>!Rn3;A#hHtQ_h>RY8pt zVYE+>2qOKu8+^v4MV==$o4dq5j~Izu|1@sATtK*5De7WbSF~cM&Aq42CU4qvS)Lrf zFU`E#p6Mg#Pwdw3hy`M;;opJUS7aNl-^d@aDVGiS@LXPtD4qPr9^ssqy!=6>xg%An z_paN{KYMV&9cA69$B#QsGO*3f>FMTE zZ^bwI{OpOIsFQz7E>I_X3NYb@KLA4);D2(!6t_UdpIpd8`LTb`RuwMz*220#+ha#4 zm%VrvXs;e$t(sE`U1+J{dw!bjG8I8cctT9v%jaqKW z<(olwvtHQ;;FNSE$HGfZhOu{Sik86DKbFC6B9!Mr=yxwY{uWs|Dn-z7CA;Li3-N)# zs@ttH%6@*rOF1s~W#uT)M@R!`C<`&5aq*NuCJv>skF&0MSrg+%4R0H9HGRcn?>q>6 zjCX2tdFv}$o!Y;=dHbpJa-DCTNO65H)@VpNUN2m#B-*LbwcZ>&`7=L%JN!HHy;jcz zU*!GMqh5GQGa>S{O0~M6TNNHji(?6B zVl-Eo9w2su8%ON0y_p~78NWz!6zq|RDxp}*Z4?$_ee{tGXOl|YZKz@FcabP!_I##k zDJ?1M$P_1yE5d-&*c-+Ic7dd`!u;BJ@=)FEE<;=u0)99+dj5jm)JK#03(kFJ5-#`8 z>x&hXaEPjG)||HW%w{pNCvxyKmcnslN|yWeoX4c+6G>~f(wicmemeT@DAt??&LwKm z0lH6nYR?>jYo|3HOC{(lvoDCjv4yk0;c$^h7vYll=5za}h2iOBM$9lL9Hpiz((TJCLE60{sXU&Aa_5ID_f37@n zY%#SFAcfIIvi2{=`t_u(Ij-}{EtzI^sF-bLILFs!uXBTJ=xG?YMg>aV@XJ%-wfO*y zu`FKD60i-g&E@;N>0W#QV#^~VHaFV@i8tel9co?NqcE3GLhH5^1>2^bF4y-6+lBSC z$@I#J10U~hwHrHOS`{zk+?Z{Na-7>d$_I|57@4W8W=K?Ac*rz^z@#~pm{rTyIS(w>IL#ufn3CFEP8{Y*>k`o7 zVB>D~x>G{=Nwno}oj<3lOzhTcpr%NqMD(O@!EoZj7P7+CfZ8omG3lQ7==H7w$Mix^ zUy8NR4JzVv2{ynEESDb7N_(4nHXOGSu~XMP{oOrEBM~ zz*%UMKKHo$ia*(Y{k5(#b!9V=bH53Wt=8XIm?g(*%a)>*_7{4177JeXcO3Kx$us92 zf*VySraA7*FfBo}CSNF)xaQ-hKo=o9=!>D9OwI7VwWRT7dO%-_1>U;qvQ!WWxPEQ> zH*cj${3cmAQm;M8Dh`F)oaE8KT@%@I1L=n`k#`tO_hmKBPd7MEDjfzt!*L^v{1-Oo zu2+VQcD1?DwK*HBik;LK7$xH~9M*@?4**Z&(`Q05aw*ta!ErDDK5|Y;YVWdfe5!=l zW?2%D=6Smm094YR`|6xqf!-@Ev2$Xz zlW+e{ek7Njqdj_}OuKrfV^9oT>x&{0kt-r=TPidxA-Rlz`6k#m$XiCT;Tx$bdv_XG zs1CR4I`TR1batG@M|OAO*~YQ-*@UM>|D50YA3**0_!k5KgM$6D{q7o?j9&QJcfTRu zyZ+z*2Sw3?3DF32Rd9@~u{ism5AOQ(C+Uu^;mh_rhj@7`Yf>A(B^-D%?|YWotJez! zUr#YCvK#USw7V8t7FA zjyc`h?G9tZ{>9bY(EU;5muA3-2Z^Oshx6O=IZn-s$m4~EZI>lbXs8DE;$NvL=5+sV z<9VWo;E_A$rSWwZ*^shDH)pqjedeEJB}4)R_(``BPQ(Wu3>I^XJp|C0s~sMD_U2lz zLgRNUStEx*3r3DdW+>JT1@)>rWvy`WrCQ25F?Nac8A7)qj$$COA9BxxT2l^eRm$nc z_7R+i(plMGDdF5i&r$NLb{GWIi(hYtdtv8{EhR+9Q}g?4@IrGjHAV|3aqGyn0_y$n zED?8%1eR`FvpyXf4Cz_qVa^pCGX#DtWi}%&-a?_Fu<(ouQSkD0`iot1xDJ&b+tnBl zfoDi~TWjqTlc*52fs()bx91#l4p5}BbS|j%_14mHSO;3_n5>DZq&P*DDwbX=7b4KL zB`yQ)bsJ^h7-mQ3Je8|r;AWN4fJs1cGM& zj}l`S5!Nnx)f1UenXz@kMQ+AcBYL{B19EoLsvs62IZdweEdvWA+737oN-&|v&)4JZ zIc{K{9*8_f!X)a~oSQZl>8-Z__x;SGkb_Xg<2>$+;mqd6Xs27UDE5qZTwEHn9R4+b zL0F=-@9(cntPZt%#>8T*ffaR{C$3V-x?{0iKD1G!nlTu-<(Ow~V4PNbLGi8C0d z^V%+B!&-`{Tsb<3%jC=2(ZuT|x88%+5qi$#iJN>@huJ2i%Elh`G6QCM@%U`wAe>_U z@hFyjM}PbP4A(h+RYreJp3-gC)z}lO>4)0X-v4t)W<%(8hr10%OY7|J=$V(qIiVGb zi?5xcjqznWTJTNBrnx0l8V})c#4@|iNj|jh`1zi6XML96(YY(bI){@`&Ut)l;$NY5 z=9P?h(MQl#cH_<4%bYDhC32*eCI>s_waJj6sQojmeg<{acE_7h5x!b8H>x($SFJt~ zgy-pQAgqub@t~jR`L2vNFGM>ZToCgCsPg>FKL3uYhOn=!MFL@fd&FE)X}~x|e)Nsk zFCil>sF`$C_K}j@p?65PYQH0>Ob&eJK(?Yv?v60*tu>oWyku7i=&n>zYc%{ew(`GB!h^)7DJxjmbR#mrV!K;Une|+ue0w8 zpK}H`(uz)1P$mjuJtZh5OToy~X<^H(@Q%V*0=tE?TmgSN^hK$J69Ui`qdS}L()tbM zmg-8IiAe=hC(W=Io;mWa`D`tPWPlQu*(^4JN|N~fH?WZkY2s6*3KI8!UPDNN9$>Lg z!1QYgmtI)mL$hPM3QWpNbz|oyD-b@rv#QvOg$DV^<%((XT;oBnnQ{ztT#fB~!Tjq9 z>h8!6Lw?i(sgs_}1CC>J9fRIX3Z=imda8`>sJa21%i$rUFPVB(Nl~Yp`n{$(HH`Jx zZvE>~-uXdR79;rAhd2J&VZ)>T_KV70o}mc|n@3o0oG~*=&Q&3WL6C(-os7gjp(xr5 zq}1D7{aUATT4c~=WShUM176}a#k6M_Oqt>mSn`ENFXaub-4gk3wmBYW9&x zqi`}`MRT8Pq9At#73#Gy6B!D?aUAtq=AmcPuk6BK97Gaxi*8E%9qak``_r60 z+I-xwb$>LVI-=fYG4`noA{@gF#da+-ajjHv-RpXBx`4}Rov6HbY-ueSj`LuHC>$2| z#hzsdm-PC1ok1BD`d9MMbyiV=@7U?Bxuy|mvNnG#*{st#CRKkd5`jn#^Ngrb2cwa1 z&KHr_OIgwy-s1QkGr7yjxqrxK?XzXR=T8dkF+xR5sAuUB4H+s7DjbnTL00hEX$z3A zZ5e4)u^+pyOc|m07I@}66})F_Uf;9|rd!G@nbUs&*ahvKLpFGYmNGNBuFJ71k(328fKbwhezqPp{&y3nyAn;a zlm#Jrj-RUPe&p4>hN35WY5u^cIx`u#6m`OgQu66e$b@i!~%>F+5c zd_w62$4mxNk?H(WK(=Cy-R{PXs~>jis@ivqUT-JOW34a}F_O~Yz@)$KueW0oqiSk4 z(D!5WLlk4llCkwhP+oVnW=!>ISJMd`yV#O_m{|okteCob|4d-QJHi8`JLAlK zy_urW*jYp@MO0lKunL6V#RF)B-Sq98at>T_LZh=dq$7p)joP*xP{KXF9hxu2>&$Ge zP2&>9|KyV+ta0@@<)kOk zpJVja5o?dY%-Cu@(yosG_VlaTd#XxywT1Rk#Z#-HkJ(5Od=-l7>SbMz!AL85y>c$} ziMbO?cS9>brtlAXV(sI9HT&06g%|aZ>xbV~7CKh7FY*n%p(Km0$0P+&BWL`pIQq;D z9nmw<`Js{j>glWFqUxUiQ4A!cJ48ylI|ZaeYN>_A-DPQz5NRpNT|l})V(Ermkgf$; zO1hOsI#t>q&+qd*-}{>X<~4K9y{~)bzR#VR_u1HShBv^I7{I*|ZF*aT4)MrwWm@Fc%jb!4THd7n= zi-UYwy_5MD2f?%)IL&&^>8qfAF)y#j*x8hq)yZi+q;;gyDBwQ7YA7cep@>EjUGR5e z8}p@0-?18G6o^Y_61{sIq}}6ZE`U1$6@;H1s}he%eb}N51#jIy=4(UL-PGt~%ilz; z;=o*-&Plautp;Yg$dHhGk7X>4K0RW%*KOIa+SxY{=K0f;lzYEvw8_`ev~6z3X16eQ zp?aKM`cHH4FH$=UK98hmjw=60_XGa@>t~V|T^r3I>mo*6bT$e-_(d{E|tt*g;%PrkTUY)j==0(qf*n%?*n{*pX%eiLwd4^{a?*laW!z0FkE2Fx9WIuKL(09&OPg5 zvcvw-W}36u$B9p}$G;k~2ozj33dHL`hM?#+7Zq1hOQ2aVs|;g*Lx`wV zmR*N5Ti`375sfjvJLZ%}8o+1cj;jRhS%=lDWluSbNfZS)25CRh7YJ%9VM2^s;f9x$ zSWu(Z{r+vljgp;syWX0vb5t%d=b1`D9;4yXu#=$y5sV|KD#(-ivzU-`_)WlY?eCmO z%z7*^PGTkQ@|5o5eLDUG*d>S2x0+5+JR>!6b}xC&`H+Mu#VO>jj_+61KHHKz_sYf- z{QJxDijgtZ@_Uk%n@{-jGCgl0YY!Z+b6hwxM{+V#K|JqV7ZJKl>=4K$F$7MXvRfn;Sw*((<9C zuwwkC!^5dhu~!Lw?_4}EM=d8O6ZW*h`` z_3GKCN-@~eNHsbuOVv9-vCu2cBX&aaDR$6PFMANHkCY_cdQVZiTxGYPwbHypK3&6@&uUjaUmaEqbQxbGm#;T8-twD-ZlXnE*}6f(xUt3uvImk@q~qr)VW}jJtDo4&yU{qpds8u8~wJoAl1~-xAt2$Xp>;;_EvP38IEiytXwYvF`vOb0h!a z6hjs&e;Gb#uS@kZk5(?|%4+I{DJH8vG#17qmh}*FUM@*J;j7w`OPhvbdynJb zzi^-)Z`>%y2Vvj?qeF@x01;aaesQ}?RGQGhoXvlEe#dE zps2*>EdyQFdeG=+Csi>>wqSlVZ0m&nS%%2^#s3gB>>_bY(e@W71S*l5)Y1NYW0UWl z|4^9S%ic|PZvi;zg$vj`?zOOB zV3VDcu`z1;lIlk0j&6nn)km_15JR4*K_-Dk6ZaYYiV+TvKb(QD%6QoCJC+m+#dJHD zx%XwJo{+Z8zNwZSVXUI7lpV{`;j#Ld7itu28NJMx$TMc5xnbc;8kFAhvUR-W&ybuw z)B~JU6{DJLZpnH7?eZdWumXu#W?~7Ox<^-K?$r8ewOQ|}1sDTaO=9eRTG46-z}Lep zNp5$?n@iC=io@&Yi-XPHKkExNeIMn^;e#N;<@zVEoczwn(5e}*m@LfH(L zM+^xIEGvZ{U@~!1ooGKY+Bt%-&D#Ml80C`c-kPkI#Jv|Ey<5*gMe&nB0#l)00xbt& zk(y)kO$BNrn0vmrgk=v|`O)V}fzPD9c*7ai0`79zUD+x)Gt4bd+>D=7YZ?u9rV66E zN-gl|^!s?ksIrXxpPfV#fqERgMj&P0;j9NygHkUCdg(`Yd%I^WF*Zb`ajlBH%0UY} zl$1Pp&lrbW+b`|Jf(;>~<`K-XJ*kzs0H)82yN#={3>PwpC8_8-G5$YPo__Z=QI*N9t>{l#%# z3O535)cp|h11>OTS$gmdga`F`1WuARpOiD)L+{XS6&W)(HN-P+rrjf0Gbv$zDk>pf zG&Iy+j@v0gbk^3(B~(*5M~m6y(K#jTjZ1FefCYa+WdRLYxE)+S?S+OIt!$AiYdQJfeprOl`7krcv>wHF77tKl)zv@Mm?wH-$8ZzM1Pf%w_uR5Ze}I(q)h{VSaxW z^Z-~Fl-GyePcd~3aL5VJg6k>!%3dWSXsB=wn$v*gj@Ik0AedyK&GN#UtCsHs{==S# z;Iq2r^uv;b_Cip4UDGzMj1B$ivA{2phzM%RMxm0{p;CJO<)I6qa;G`po zqXB;{0wV2_*5J2EgHA&V{*tv&OAp|vuA-ci*ukZ^>OX-jvbeXvOvUN93y}PmAG3qLWS1&%;xfoeyX6$Fv#$&h0+0ic-LO5 zaJCl7_xW-m!4le$+4>g-3A@0N{b5F|d?;~_O*W5J*GXro&^pZjU@$`V{7FQAN&J=9 zT=t@Qvq3`^(+r_(v*{VuxVDbgC}+0bdVj^=z;>T!MH_~!ggC;&nu2bTx^yhbGwvO@ zmYYK0@Pr`&E0uUYHgzwB_DXYE^v`|Vy2uOQ!gZ(rBlCo=017nL`MwOmUk zdD;?B52X94;;-G6Ch9A-JvZ&3k1SEH*jQ2#cubjMD^MssmI=k$l_>be$3=+9oE8@Slv0$=cTf z8O0gL+;r$%&P-yyqXXJ3P`d=RcThs5 zc~jjJMbo`2k;lNG((FU=S~{%>4|kI<0|k8GSO}jNjb^Vw|1ic`Wk7h%{kws+lYi_R zOO)a79la0Btlpuw0KLbOqJvvOv07f8<7>*}0L!ql)Of#0C%+a4?auB$ln$pooNHrT zy^=R?e1Zq~$`?E=Jh>bLqX5nfR@9rn7@Et_JNoTt0QvAy1=VpA>Wzo$vfS@>cJwzO z#bIYSVXXKYwQEsvWj*=M%05b!@}9VZ`K)RW0Q5D}UrDoF9v#KLkzDd19VIl` zxPOS>$%?N++YH;MM@b`wm5BTfc!}+ns|)eQHu-$`ald$`*_`TOy`vWrJ%gWM=#w^6 z()g5Oyrac48OBl7tNOVyZH#IXRfyKX+!Y2_Ol?Qs>PhR#$1R3fny)fK3}5v}*n`eN z10a7{B|3!+ager5BkH7Xu+Dxa-%=Yly6G#Fh6_&Purw+b z&{^*a<6t^JmTN3!jrJ8F9XTII1fzCCd>MajB&97PaO9qY$ycV_K5Q~Hl zJ%ssf#0I_Wq%xV;!a+?ns;q#dqntt=_L`7u`KQiIx^X>JS?_D(_cGUBVRIQ|6S2SN znS9?1`N`K~nQIz-VsuQlXfipyzkum~u5L9JxF7ZWKBl!d$cT74%KpW&KaSb!r(=8% zwcudo#@1cDld|t{yo|w7=`#6wfa^Dn0?dR3Cy`|3jIjQ6urH%#H3qR{cUb{uZ8z}7 z#HXJXxZj=w%?}s^`?RQ!h=-^gG&M(n@S>~C53=Z5#BOf4jP8?&oWs0PXO_N zd?z_a?bhdSo5ZZ>=t0U^$B#bEW>_i5bOCGo-xM&k);9^++IBLhg%T-J2=MN?5G*!{ z%Txq()?}K1sFMq$`8q`sgg53+n8j?;ZLPAb9}B1%%(hLCtRkwyi@yJEIv$c#H`Z#y zK*~CZjM;U`&=(4m?%e{4vd%=+45Y?Ky4o8K&-u94!%Q#NKGIn_GPaU>bj{gN)140a zUSx@x4heE~m=&F`5`+y_QL27cZ#Brzv|NdM9iv)^dh?7ftK#mT>CV@?y{o(D!&sMK z8FO|WlA)$Oa8RWVh0?pupTmD~xXi=p8aW*aNB6@?dXlo$oT2NO*8QMdnz$6J^D*d) zpyFv*g*Q9iLAd;yrkXSLq|VOMhm|q8jj@sXHlU{BXNw(puINe7EDs5om@g*1eiJCm z4b{JS$)!}e@`9Fiw{IroHjj^tKbv}J|*D09s}wB#ktEpvfOb{ z{!2XhztrO$K?yF-T>~&V$wLa}N0fs9Kkaz`-W`|8v1Zo(mk-%P<6{&kg1kK7VWZBU zA5tUDyG2W`fTj3dqX+Q z>%g^USQom@K9nsn;*rrObZ6hsl68S1X5h8g(b_Ix3LSz(j;Yv6{Rp^xJ5j6~c|25o zZQX7haiW>gU#+Fqopo`#HJU5$e{s6G-K?Ep?W_Jv*JCwV-uA@=@X!82qi(jeGuj=R zz)(0gR1_DIdTywS!X{iS$Y)H@RBQ$O`4KR&{oYQK?bAs0>Ui9WhK%8t%#NQL)31pr zCjN1|i}2z7EipFAY;DM}%`$%Jn3vcH6-|AspmSQM#n|a9`Z{B`T26FAtBc{tYJO=| z{g~lM9hG41<-sZyaH_RwR9?QI%{gK0k+sF`k@Ypfm{>I~JPA^} z`9?7D)!MT@+3tQe0-_oel8y$dXeP!^Cu(<|r*<5wXJq7fW-B$~k#+bfjv<(>CSG^P zI3gTbsq@UF=bCd|TB)$#Y{VmXm1bSz!7s3Da@n*=kHL)mqp4+~&APnjTCcHh66m42 zUfUDJpf=w>2gzI>URIKw0ak38p6tV8S?SLOd4+m94sHhDS5+nI3L&6$_C;BfSPKnC z1j)lQePO%@iMEJOs}c#XC)frx?x6&`M3aJdY5FsEiyE{BS`*DJx}-I>sfnFn+&74k zpBA4qnZlPBrDD}-F5=GB*`C4@o_x-ELU}r-efBXFNCpxrMM{~l+fCYhORT;IVdSgA*?XF^FeNN-|qB4gA5kAkG63m$S30?agb(yIQM}$XCX@|DW zMEyr@q9zbX9umr~tG@b#>`YMt zFd=?tM11R8Jz$yj@MYlZof`rU{*@@!Sl`%W_AqC%2fKODG)`m>}qk@$y^i;iik=YZO@F@ z1=324A}!@vZYw1to%w5#Y!oYvtAqM?IhKU6qci2cNJjx$G|(u*zne|HkNZMFmXp&U z#Hjih<)Dg}MH14Tf~s&ZQ2)AQG2YL>iC3UhbysIC#qIw>>eye@e&~c-iWCq&AGk{>6Eq&Tmgt(7%o%gs(hgaMGpVq_F>V-yn3{tILwN%9~Kxt(Atedz8H(Wl-cF^c|ZtPO^`rxACJ`A z${dXHO$;PbxEBnvsOPtV`Cl$OnPDS`{OKh_q0HjR^V(5+;C=i#^vyIQX;=pBw;n!a(6P;Qz!0dkXvce|pnuoi_42A4GiE_OhCPy%F)Ymuf?zIrF1A z>+;vHRcSRbEspq{t!Cld`l@|>-E}E6F!(!OZaCp0EoA(^DL8n~mag;GV2w`P?ZC~> z)t)J|s=P>Kvhz>$8F&6sb^bhX0Mw$QS?iMR=g?PEp0I?i;DZaO)(!Sy()3lh&7k0a zXT4VYdnTdO4nSc@|BN)8R@)o%JBCTQ&(y|Q^?&!V26h5?@A$)o0JLzz0wRZGnn}nv zEFcUfmb_h21_zs|xQIcaU?bDoTD!WV1|Y4|hv4M!(%K56+FB6kfYL>*?}O!UZj26` zS0bT5UI!UYb3u1x98P29L9j2*^Gj@ho-#!|=Ji0_7ejj(4LEHk1R?R&e)cbp&5s|j zn4%(#5gKV!1%obO3$%!^O>=Rq0uVT524cv`2+c(0LfEHDsgf}|2dOot!+a6$L&oUX zf{lzJnA((#s&8f7(Y)m<=9v7Z*Ut@l2p#*r^xIetd=MfFK@`u5oxaqVrH&RO+ zspWbCEv>bBwIoI1@uK&u2;~NV7h}w`U|LVbL%G?e-NWNcMYkwF2U`Q7D8Q*<`E69Z zLS~cYRJ+He&d8)$^JrkmslMrKT?&tM4-fw0E&;+RH@;#HTqYs}*s`Gjtbu`HmEEmL z)vF@nW9c8V-B*PL4oWJT;W6)pYAW*2gll>P=N{3(=xMFo)?sGHYvL}15(GMFRS&IJ zW%*z?kNRWGw9{sd^-UHCMS1Y^OQvb&9SMt6=y!zC#y3=a3G_@_?}6-TXryUfj!FTg zFjssyl(|otC7p1Hv&{m{WnjZ<>v; zVAF+}BRxUpv)|&f$u2sd{^Bq?6o}UX_E$-_Ysl>H5&#ok5N)BwNS^yoWHqmG=-4dH=Q= zbn|QdmRLh<`I^^cf^htm8QL>FN7nmzZR_;nJfc9P!pNvw;Xduw*uJz`xZj_{?X!PI zd0jm5vj|M`J2!*^y!R_Tw2kZZ;`%mFAvcaTo6B&FoPv0S2J&mGyBIMtB=X+I=FVS# ze|Pe(G>H!5CIs=yTp{f^C)A8o_wrZ!&B4K*o}<{hsVbhYX`Z9`hi8;|k{THE58ehN za;@&t11&9k*}wKR(;R(gD#A(fld4daO2xxUAtL=Dv02R^-fj^Q5n-G1f>O?}nz@+! zP4P+?Qt-AfbF8L$sfd@~(lxK)&R5OsgOGR@4E(#b$V}BMH1GMdK3l{PoC3p-Z{zh! zJ)9R*6Pwj?)i5t~S73 z`hx;ikfnigb*FG#O4rc`h;I ztHo0g>x~*8zbf0D`MNP?u6XeS)aVJpL=$gx#|pdi(ciPxB;J;9kgYe}C;)sJ(wwIT0tx}(SeDsf;91G%i_h$VxEeHVHroA<+_$LO|9BmT@X86i zGL$l>+s1RK%Jh+uwzpBrE=AwdHWfRivJ&|CDDd8LhQ^AObBbj* z$mfN|u85GY0eD+8r?fdoO>Wxwo#t*OmA$m(_ses&REaPzh3xYs(4TiXDO`*s5Xxlf zv>=QJo~c^@C-3@rn^^;r-Da?S+hvH2XPMESHxiO@z#nIJUQOy|25|w>R%dR?e85v&s)V3TKT4K){NVf=zwxsT_5)Xe#pN5Z_52 zkxkUOH&}*onky#@v#peh&%tZzt5mNor_Wb$7G}w)D(bGWJ*p)*Dp?D`h;$bZ6@!fZ zOeUttuuFL4_R#^U?%G8;mOjbd3|vCKOZ=(K{poN#xAtkS!i|PGRlF*608u9A+Kiek z#dG31ljCNa(%-L@^TU%1M;*?(Q;e^nirRW5rD{~olOB0$A5#ld;HUm0j- z8@9A@DOLLoGG`Hdx}~svZ-#BcgtL8Z+sPjH15I;zyon)sa_hzQTDeX`V04|CUE1_> zEMlY~aUMfkwBJ}Q#Z?avbkG=o+m_1ZNKq_bllXBnG$*PG-_s%d<>2W z12F6rZ%WRea+X_(`G3$3?>`&c2X}(=;PJmlD%>Z-C1Ji}fhA`ZRx-HvKo9(o;;zWn z&Heua9^8BX0G@yF$bTYmceX)IkJ*2q1{_YGArU#;{hyjM|16(hzvVx0v@{#d*=Y5x zkQ4ns-~-zIpN8mTpz@t5-m&0tsG0FjcRZ^n)y~}*{A3N@lltD!OVy&KHg6io9VaJ4 z2ls6?zIqGshNStSL8gwca5ELk8URZHp&*K=>R$}xe(lslXr$k+bQ8uVLKH|(hR&Z; zp#XB`pE#&ki>b=KH>M1WTIYhVQ@|ZzU1*3LRD` z*=<-1_0t?xqCkMjGhpe@UVwoZ-hzP+WxbI8KMxnE0Z|>VDh|*5aPyi`DMsZF#as%X z_ZXYNS3`Z1MMJCj*Ot-lk#$jwk+r2P+0qGR{ik%*XV=oT-A1b_VKBO!AG0?4?aB^< zL);4xGrRJDy^UPRWJF`>O z4F=m+q&ZBn&F+7*ioBbuMFt#ykbS_usy`jgy5CSclLFfF6vqJG)1P#l@TnxMtm&s0 qPEU_Jp((>!UCO&E^JjNN!wc%*`qpT3M;4t8$KBYO1u=!cbN>&rbQ7E9$C; zmp?8X=85z64bp&rxjzVp`C>KT4$5WFVfSMC$4(E1?m3mlRd> zmHvf`4)($a;)25eg^T?UF7m(P{x*letK^x{xKQ5+99BOxI1u(nYE|F=&WqxIrT0%< z?0@G4@n3P}$;`2z0745y0Z|k{2mk~DQb2&`BY*@le^fvq z@IMiN5(uK8q6W~=0s$0c!2f~($Ph{@YA^su`RAenfGB>0fdB{@Af(7jrELXgqxJ|% zK(N!~3qQ8z(1HA+KpsUw9{XnwAOs}DN};IDMrj52V5d5N4PYdLK@bWEKnrmEP6$S% z1a$#1)E`m8B776fSP%k8ixP&Z5NRhUu{0s#Llns(CV(xvF{Zc9=$(@07h8B01hcO2 zV$A$AvnXck?^2lORP2kYcCGjgm>Hrm2_w5$eqVCflh&2Lm>Nq!GXs%V-*Fm_Hu@AX=aL zRdV903`Bcwj?ttjTal7W(PO6!s`fFwJ)AN&l()Dbeh3t61Fe)L>BDCR`dLDq6kW>Z zUNt4ysp@T*T^A+hGd5efhdGa}Adra!9Wc{xrXl}O4T7TJH^)Gzf)3auiX=jS@lB>7 zfEd_E?S3L3S$PRi;K+q7+6I44o;tr;&Aae-?0y~BHD>eDuo89{rF~*g+S4SX)a=McHW>jGnS2Q@&APWFyeOeywtqkK)eOG8gNLN2* z5jX$`n`_)WJ@^eN_h92;5M z>vKPaJxY+bZB>_vdDGb}e5s7Gm_CubX0$p$DDcT|8-uC*k@T`I&d89B=C|2f zGh?ApKutI$hLaY(OVYMcp>y026{JB#<*=p(Ib0$L*BqO|2_0KTG()JbykqOcBnub! z1h}#xZ97}lfVEE^?QF6ZV-QF*-GaK+x6%T#pX)sSL{H z0DM!PadVV)9$RlU$4cgxe^`0eWyK&qdTJ^N|1k0#91yMRfSuGrmNH!4eQ=HuHh#o4 z=Tp@y$ANY%H{Q1W+F`IyN^g5;(-rIesq`Fh*g!onGw>|0=%YAm<)WfRSpqj#LcC>n z1!9+oNZ=)D*TNuVX?4KAEfE5xMuT9o?yZuZgeY1nveiRs@l~q|7JP0XJZA z4>SEbsluU?r!0~-7Cg1wtdC`p{q_zL=_5Ys^&l6v+DA*2R)xWwFa(o~T+3*$gofro zY0{I+ zuTM)PO{%=H#~R5!=vv-P&GA)y!QigZlT_# zR-!yUxl9Sv=$7uE3X|7~I#RjTa0%YXEa& ziV@a-IU0z?Ui#_@A(f|=PKy`uog~)S?Qov@S}c#&ZD`2b>FbN;-@0M3bg`(fL>~dz zDsz9EN!1gU5y~nJh>w`6_A7T4H?uFCR&9CMuRkocR{MZ0gVfwSl~UVjTevYZb@n(j zwBdHGqWzP91TDmGU;iC+;p{v!^m8{QXb6Nb7)Jp)3YaFaTKZjXPz${@?qa*3O(ZF! zDNc+0P0cKOhdu<9R0>U`oxQHnVabF$rG<0yN*J93=#q9D@|lswv+!rEmhIYt@&t z8VBLdXj7wu>cjqT82mW^Q^a!iZsTIBuZp9|opBz0iPwPT=cockNAW#ZQp;fL=h>?{ zW*b#|nuCGp8AW#v^ZaNa9mce;>WBR^FgEkagOvn}o4L1jmGr)TvpRB8dWVQySvO>DFpAZD)%xsB1G1=6&MnR#u*BYJy z!HpAK3bSSllU&^I-Le7Sg*kqxR@Tz_Qto$K?E*5RDil5dlzD2{Gv9Jg1T-*;F8i!C znoyIzsrH%QWn=gCqI-{L8!vC*zGgR5hM(5L3nGy`DW;@*VqHAqS9?Bo?y!vJ-{Z|A zHc14G+yF6e4_QWVf9&7kEa=(+VdwO<|B?3p>;Z(dTilWMMYFVYD6*qap~6If5wO!3 zc01IMGBNg>Lo?l3u7xZgAIh)q+}-K^h&;tR@;}IK!Rc6K O+G}3Ndb)m572wo9rP!$>??3XYpeMo;H(u$a1Ru#3-#enoP zeWwzwD2DX26J+VBsRjjPtRZHv@f=tim~NjjgTS@L!`@Y_A?;5WUuouk?{A%hp9)8v zTV#})%RPKaUb>x5Q!|gF@d>9^bd{gsya(Oc1J)2yJywWX2aNu`EdujdG6_+4CU(~o zP{tr~N?E6Y0iO{+*e)EjjfBbG!z@g^jf-Trv+Rmt>l!9)f0UJ}5*9uO(Bo^@k(~{a zsc-B28l-ttxAl30inh+E>=FuXzm1J-%%)Y&_Rh($~w>9dG9j6?qs~@ehr01*) zyVN&p^uak{?`pwu!s_~UYfktCtKxFLF(s>*dk{p@k*#k>YNBo=?j31OO(6QsQR4kN zD6>ekoiuZGwWAelyfnsfF~VHUcFnq#Y48bg@P>M_t zgQE9EO->E_w<8tT6^;SH(sIo~UT!&gEXg2F&VhA~ysR0s74z zs4%k7%g}Ux0;x-#T81mpI(*}0gtsXzy{+godGn_0s!q5IPKR9p1l(vqPRemkQJ;F|6G{rur!dahz zG~_usWiM7!xZ4$iG^yalPR=J3`envTRZ0-Gu2GHA7htv(?7z-DR>|niMKXr_~ACtgJ&cXEYS7Ujzb|LaESjUYIB?&21GT zb8G+TnRNq?dUVDdWh~XxSG&#m%Q%WQ#M z4-K<~&{|EzNs80Vm_>hSHkNv#ry|=m!fhN{blh^1$$K!8p~*Sdv8}rVkyeq#c|GSe zBcHmm3BGiRU?Q@f){<&?s2W0E)=|JA=r%vgI%@;hStP{hw9R$U#rc=qK$Ag6=FRyu zSDKm2IF3QdEn(EI)cc!#!^EgMr_l|)YKomYE3YgCz`~`d?(tq=k6mM@uU`Jz4#{z5 z+W^K+#)y88YI62c#L#7f$pM+!XO04nWD5*$ z8NKFP*t9m0WBcKGA=Q#mjiMTi7w44)&XzT35gHMr_Y>s3Uo;)gG=J6IitUB>HPbi| z=5g$-7v0-6qBxkOxTD0DeA{uaH(QmW%9OCN;y=y2`rtAG0j!0AZ}V6(Nsz_2RhJS1 zF9DI3EVRrhiMQ~S3KeHzFqeGD!l=(cnsS(?t^Hqh6PlY=mk$crFu9jyb{%_3IyULK z*>~EN$vj!DrG41O6 zw1LpRl2FibpE|EZ|snM>JbHTYos39wg2Tc=%=@1lSk! zUb*E-avfTEVM6M5w-wVj21FdXzJZwpae6*Gixm|LUE}R<+E1m^yG@h(wxHIEjK`W8 zcDg?fvWhsuDWcJ-aHVt}k{+yA(>O#TXo@6^0c653S6)!`i4U2euF*cZyOUpr``~9g z&m{9YT4|>wi|N;hii&CtgXMeQqVj-ZOX>8u9z!3QYK)Yh=9<8TH4}@_zVUmN8xJtl zGz-3bFXc2_=~KgTuq+ohy5inFH5DB=$K$bSX@Ts|f6agS;r?}EZ4A9LpsBoe4j8Ob z9q!tS9uWe}{m}<)S_)c>7bcboTv4#45dq=M8aQqQMm7H`_$DXx-7$RUxVrV zNHKx9+3Fkf+SHS=o6LFE=@rIiChy#E?7A$*3*+nU%$#Z;mM~Jbargh1F105URB4Uzd8eCL*vCs*Kt+EvubN<=fn&36Kz$? z?Y-j1tag6N;%?ss=6<4*MZAt}{2EF!%gdX9lJ?rDrn;cmv3$5@M)W23>__Xm?2a+G zh#7lDo|roNS;feC+kIa%PxGJ2?wO};GA=qtHkI$`4lPRJ^%?FBbv>!HOAGA@n^=g` z+WETxA)5gSWW}Ez`^rB~n;g%boY>viJO^y|y%!KMe6BiJbntEP{f}=f40krj&ZGmZ zBP*@42Mxbl4T{<$M455`CxA+R`M!X=pBkjiGqK!l*J~O!oo?TeRe>1v)sojFAx7y2qHpc-iyakTFXRYL&Ap7>Kg@=%8E|sE1k4X9FN_O?N-EyX=WUqw zzIhI4zIRx2(tP}@FaEvO&Vk+Y1KNN3BOBQt?MuzrDU-0|i0l|mux=199({mkdn|HM z%()^GzZ3SgGIOS3DLP@_rY$JeKJx;1+M(rp>-n2T0>(E(TSD?YV#kLmPHJo~d}&zs zW{!@eAHN~-UEnrCvuV}?bxTPYt|jrk>4?Y}qTr$F*qd>2L%`he?&#+kHT}>C8_!KM zd8_Z4VxB6UHnGUfrpo~g-ubkpTzJ*iEbWCU%TT};!ZH~>;)tM|{N3s`w+p3zO~2aJGCd87`O@T(n+qS#WU z=dap4e!ot6z5hm@zpW`@DKt){ZBp3aCF4}JaJUi0)Qn$CG2XOJ=r->9#F1{k0N`P5 z)50q0j1I3#K6mPiTUeKKinrHxuj&5?PPRmsRI0&5$fqJ16GM20{r-h1 z0z6#0uxVpAHs~O++-M;faZ#7~$5d}JQ*no7;|&(UwGO(yZM)Fk$)B<=fo}Ya0XAZ7 zZyAcNoqis`6!uy*KVlxxj8}0UbZgC2+c!+PU~Jtp@sVF;SH48x46_nqVU-r-id1+T z7UZE;;ReEc@9MrgGcK39<6bI9G3x#58Gn_0-L}aYg#|B3rZe!02G@4kcjMBGiX!il zgMc8RqNDq*qGs#NcGcdtJ{|nl^%sAR{j%OEI%RtMDhnI8GM;J@5^HBJVCHzBNHY6j zQMu;Y_o8IY^P@%aYp>ckE!nUTev!Y?ScWWn9Xc(<7;(5i^9&dlT%1j=dc$G5FPV6pu$m=o`C z86U#N10KNbKfATzlHvI4Z^+%2bHEMPAH2&m$hESEkL|gXzm31WFI@5VvUt=;$Jq@B zHsO(TfW`IY1v>jpA;aHUow|r(cgcz_(L>#mg)XUp8Rd#sVr$JyNt@Gn5r1lN+orY_6OGB$d8~e9FHd z5V(R{cx77p^t&>%vosT+>n_c%?h(&3eszhv(+yRS{(8k98uW8$y!Pu4H;f;jVEvBO zMN_UouQL_!Ub17QkCRi+lgLFJWPK|R+x0X)F-%=%`{=NvT6vSGOxawnA2}fPu(veO zDH_t7l|n5a|yH(q%}FeG!C@7JA2k3X2rs}{@U#jj}pJ@ef2$iZl_6X>AncYQhLtN-To-USzx z3eybfHgyI(t9v@7B!w|K;P;S(z>*0N2|5sPf*A#Uxp8_C5FvGK^K$I9qgEFXm(@wi z}n+ATf7?OIQ*`D zMNk0v*=i0oS@WN#>KAVaF;=h&e^eH*57e++q?3D6EI-0I0iLpezI&_IxFcu5Se&aQ ze9|zj`6gE2p0w0JRyTmbYTQ~@ZbaFB+?~T?dSfkqH?EKG>aJQ<=={cX)!Cp0_d)UV zgQqAV)IThfxuw}KJxaSFcS7X$?J0sS4QO0&+Wf-2kY4Jv#NFf|yaIUsiWV`eaE&&t|HZ zls>xOI`YwHLa8h7l(ehx^Nw0!{nuS>9(d53OHM@$-}4qy<8_xP6CdI+cSFY4;CS!9K^3N# zxCpAOe2gc3F)edaNw;J|32Fa!R)oDBIGr5`Rwai|8cG^+P2R#Sn)KtwK|}va`}kk2 z6&SvqgoD0;n3sDlD<;4C9`3@OJ+wcvEf_wmwj|K^*yMLR8l8&QMXSNlqxpXQ8%B$$ z(#aP{tdcBf4%7t2+sE-Izu{b#+BzQF<&3!b2(1QJvw-dn023GQAEUi-h6ogMi@DyK zH)eFCcR$-Kare{A{Bz0gkIzDLni0*=;+ewh5uzUt+&MS}VV63WrIOk51gDIPG+G}% z)=O+wl3Cp=u9j4KeN}#7hNP4@@u~u9@*~5qitf{vqf4*Qa7Ol6LUjwXbE}j?DjNg9W^BiYQK9`U$VX}t-`Tf#O>6z4- z)9G|*?@-&)!1yt~iJ76t0jl%UJpCtf7PT>hn)gcbI435wZ)hTOs41NjS^c?ey$(EW z>~rgHBkA*4rxdb{3a~Sg{cz>!%t9|YF>4W$ch>tBd%Acj|M-d+uFGkX_FsNVZ&p~u ztume(>6_~~6os_bFW&Pbz5A+>*$Es=GT!8;JV+^kvNgl3dU;9o)SqV(-|4 zlyBJgm8}ytd{qr%^PrHbqQ#}ur2q`k8Z^}bY%*@Lx^YpJqxe$mUzH;4NwV)Ajb(}R z(~`@>(H4=z&n%YJVohY~FI2)h79!1spPd6<@7~J1K0YurlYi0CP?KW@>vi}O|963_%yPZ zTt4;@p>ggR)N0MMhGy_K2J`S#GwMn~E-4aT={YsBDZxTytQWi#-_k9CGHfvAS$Amj z;Ui+>r2doCKi%JDK#gM_TXp}{mKSH0*A+)Yhuq;uO{*A;$u*O=nV#o> zV&yhC=il~ELh@H_H>wRE?)oX~zHcROrwgf))~apqns;ReW|iPMxjsnyRUkG5S8Mh+ z_1Z77zAbI2OVsLyF3-Q9n4DhEGN&ZxOg(b~Ca+l}U^5KLcMW(b`RoP$$Jdz|^{@cJ zf(HUx0vF@&(EGyOQz&(a%H&USIs^}Gp%#*mf&_A3Rm?~S^?_gu1)FPUK7LxMeyGa9 z|E=|*mYdV7LyYGzbYS(f*Yma3gZKE08T!)VXdlH^-|HNn^DZ#m0T$x77UM2sodxP> zzKQ*mi6;>i?tLc1%_*-Gty0V_E*Hs=40M{gF?rg%zuq`^dWxMc!RIiqmkYwLi!zB~ z@I3}ui~PRNWClLBZhg40e;`IY)`&?P$EEF9M+kV(F)Bntr^*=f7JSH zsHE4{N!Q--XmTds6Kg)PUs6C@gl7ns<%Mr^73BoPiak4~3sK5A6>_@$*~H87SBhr5 zN&9$C#OqDRc+nWX>+10hTX1q65{rTZ8{zEGIyiEVkl59jtPNDZU@De7NkQaG3MJnN zgIkDfLzb@fp9Yb~1N(3B8b!@SxLemo+h>1F-lD!;sAzsOG7@C7$}keGUs*e`cjv3G ztev5#u$%wlRrdi+pS z1tiTVWlW?yj$`U(X9bMK^NwVGJ+;WRPX13`6?^ID{d6`rwBIatxGOVgAZ&_^uE%#F z#d4inssMrqT{~qjESD&Ica_QCQxKEstJI_(;JTQfSG891IRGaAeT0yrA9i2fbd@C~ zwxJADViQNMBt%MH^TwYmaHT7%&4&2 zQ~40;b>NxVUxB#1*HPi|Rqe7@_zM`dog>sX{5o1MdJvhLFBII*Zc2mG#Y>L7W{|%$ zrpoCw^qtBaQ~s9Ujyc9hF`$S9_+02x<6E1!BFlEmp(5OWtqD*aKKCue-^GsbS#ZJr z?x9ewNZdS;W(s)@Xr)?ixC|V@G#d1%e5HzC9C*Eox>a;d@JLGjrf&9Cx@+|z;sg+BZdr>gr0=YUtw&War| zLnx4gzJFne7cX2Vi%W763PzJ_AHNGiAl)IXSQnv^7dDTsHvkc(fMEKhP>|_0O||NH zshC?Fqf?9cyKKpZsvN_w((0Zr4(=j0OtV?Os>z(>M7MI%w##Lp&H)Oyw*qq8%II_^ zc6UhlG!?9xDc@geIQ2Nxe(%Svzo1#)O@HLFWr3^iOx+#-$>|Yj9RKpQQ%8H>;BgpH zqP*v3dE6~2;6qH&neO1vvU?@qqN*8EE!29!02llC1KRv^RbzQ}m5w+>l2vrWp+)3H zTqO5|diEf_Y{pn&>rUf1@VE46-SCIfOU}&_(haMRzF5g zPogpj`;K8mYRG2u9dCA|Z|mxT1TUw87IOBigqjY3lB83?9GcteogL~ zb)yv&Nt%mjmRW)4y)RZday&uua}Xv^+%0_Z7wdu+N84z@C7ocWj`15kQ(%+8b&M3q zEtngO#Jm715fhj{=Ow82Gh3c}!ct+-wAC9C(6RxA@rM`vJZXa&hW2z9)b> z!@)vcl6;ok&N;jV=gV>N|Jo+_J*1thi#7QPuFoY;#?37f`c>+-0 z)enK>znWV8lHIPmd03(M=zdOO$S+Y{69<=gPpu3&_Qmds2r-g>TqkXQ*%^%7?73F$p28|aPq}{*evlAe)#VU2PQ_^I) z3u)CMoO?^ODW7RHK}g!nt`KqMS#`OA;f5k@ZA{~*yJ&ZmR4s1VdX^!yjUsB`zKvK< z6uUkt8Dr8=_QCe{r3O2wRv-TBAgg=Y^P0SuZoZu#;&Izgp=?FEe)lhJub*9-s%|Kf zWxzPL0XHqBeSb{-xBAT+LhTSeF=X5AP zBCPgza;HRms!MY(H-D3&5!UJacXVr{?srqyp*s{}9YI`zv0nE#4~yd%o??wfue_A) zRPu6X8E|}6Sz>1|CLo1xX=V)YKmeB+k_?LyRso@vMdERCvAHXTNz~2(nIa$!R^qrAR+7B*86t4 zG@i>rSZm=NVDl<-6}ey*IO5|omd>+Ro`bxnBU{Y@{dG_o^=*F0N@F}Pt^p(#HX9!1 z8b7CiiK^n81O`40J(`N4MYdJO^DPTsUqio)_Fh;JhN2&4$}E`V>7RUA zx>JVs2r6Ns-}sUp5&P@r+!_{{#Lw>K>y+EJnZ7WUxwYE1Ei=!1Y~vNxH>O@Kaw0C) zOs&J0n>i=NF{U*gu4@Q0Z)+?JA@YdbVpJ;fbPG=-l(>3+)GMkcN1~gvBK#1{>zVZk z2(z25mln?X(><}Lf|)wYm+_?pFh)a#M?DHaLD@+@Dy=fIS(UteTwvdH-%$fWYhfYr zQ%`rjYNvN&@-R>9lhmA^@6)hIz?z^3!`&M!S0E&y4sA7>-LizsZ`3TaSX`lc)W?uA zN1cssr3+gk3i#U;@>>sdhAPjQ$`@5ZcjbPWvxx@YYO2o>c8Lr91=W$8mah%|`p`8& zZ$8^gd^T)q_b!k5t&TA8HlAzTEQ0?F|KC<#4VACBZDfC-{r7t6-2tzq%&NsNDTwSO zr49Y)qpuU*>s+hP>~4M3yKW2?)0^}5%j=kli}u|r_V!m>{NDbFihJg;IXP&uBSzHN z%ng4Q>jj3s*fn)o^`y-yQ_A_=>N|a7>)WsA zzM}0~ZP(8M9O0=M(s%R5X?$k3;Hp4NJ_dHED1xERT~o-b)p`_R##BhBZJN17DXO3= zlE4orCsJT9*NEA^BwY2szH8d`LNV=k8CA?q|HuNiUdM;)uwn*)BW|+ z(wB6Pw#r*hHhCUabsS4)gb(DXe5xIOT5g!3D%l#n9hO&s9jR9Sp13znNZU~0`iZ@k zpgql&{Y~nf9un`_-ubCHS%mQ%fL=U$iGWr-DD1LKh&*UY*)<&<>Rk%0D!!0@e{>|7 z8$piSyPSOlZecD?4qdbqU1OKjx&>QJU4zw(f$f-tT3Ju1j{o$pX(h?0{(Q?F{N0a4 zVp^jro!^_RTt4-4DgSjz^^BRJHYl71L~xS?VY+}{9eYh!%WP*Q3s5#x;Sf3$vIrEa zT_(9zW*KQV<*TE^ILw}?GAgV_`}Hrusmkj9vlw{;Z&i5T{j->O>6NRxDNMeSGxu17 z`MtKfD;@EOc7b`d z20vjq)%|&*EEc$SXs0h;csx-e`5FGnX*K+mdqTyg0lq0n3*h!CSQj*xiBs0u$8i86 zbZ$>PCi?oN4%D`SFLg07tphAg$FF&J84b0||Ku1F!;-%z4}7Z}ItLWOdS?cra#qpy1m(lfc*NYegl2adt$cZT7^oT i{<_h@K<(AqjtAPlr-)yck~TzI;w3T3E^UMJrT+n~XV0(z literal 19254 zcmbsQV_>Dt(gq5zU}D?m#I|kQPA193oY*!ewr$&**tTt*JbRwK-|z2tPUrrSYgJ!e z-N{|mRn@&dmp-=u2$G@_q5vQuAb|MS58!hZAOrvb2L9*yYM@`wH;8ZFKtaDjfrEoV zz(B#kKtn-8!@?mUz``NIK|>>;As`~7prWF}z@uZJpFgV1Q6zgjl zApfl8{~8eR*9yQsR{$_yBwz#(1ONc&CIA2A|8IXoVPL-Mfihl+FQhA^G5)_%fJx-y zW;PsPHOV-+5N9L$kZT5-P8XLeGRIv1*9ld{b<*7}F;B3;R#+(2S0h79duBnzqfmHR zaKZY{blC=NireSk-cFndZl)6fpcsO8FKJlm{X zWN!ZL?--o2UzD)$A@;+FIfIfld4~pXJ|3Z>VBsk3`-%as=LU{YslRxk}GWV|}nj=#Lk)wU75> zyT9`Tx)tNyAk|o`(B2!>l-1#q!}oGuA(`N9F`UJ_O@|)T)K#$(9r}Gyo|tJ$W16M# zOV?E5uSWR9tJT3B0`lLQw=`AHp0H(Ed7K=-4?5i($4D0Z6bqU9>$+SEzvs#@37@YT z-I3QF{<}i>@T=8Hl{AfAn)8+yI{J$x`vpN#83*0?Dhc#LtwYxPeVqQ?f;`hd2hEgn zMn`>19kTp?Rq|5wS%WQamyh%n{b}KxKWUtjNJwSUe#{>e&Y(3JX~|Q@Ucy|L znOoF;Z~sE>{FeyuUj{mRVS!urVgp%zDSremcwBh`k6g^3vr z#x#`eo$jQZ@WtWZx#F|AZ3^sRI6e>M-6p%)bl$k5Eg#d8$=EOw$&?YzC0k6Hj}=Np z&8Po4h#fHdX4XJ?>eZE6bNBZZ2Ewbm#`k|@x5=zp-g-#)gifn9mSxFaxIIaUG!v(3 zo3WuuFNshw{B7Fd(3ieCC5NB;lKxjQNI5Ut74M%_)#F>c&0vSoll9Do(1PRr-1D2J zwZ?fj&tKZkccZ)B97j&G%BxvcqpR{^q+L=2-9(`yd2_; z&!S|Q=w7zjRP3&n^pV& zGQs1o-7_hBP8Tftl3GU!E-WnA%F7mnC%?-U$fhz$fm-u$KU?5Nw3@9%R=Rc2E!=ef zOZ2GdSr?mF*3&JoZy3~6C(=2NMaYve<)pY9Gy44^@_d_^&=FX|TEF?qz25X^=U*1k z&YCAL-^fw#SY3y;T}F`*&f%gMvd6<@Egc!L?a_hea!sR^;d64qE1pQw^S>m({LSn> z9oynT22;%!CB4j+5~@TpuoB+!E%!uW-Z~; z7vEDes#U6m)dpjCrDrX<7KW_OQilsY9U_0*-@{uDPyE*>dTh@Q9fo)BpDi`NL-2Z+ z1+ugBYLlvx^jecsPiiqOk5{0KXr4*)Xf50vobvxw1eW9B<$1buNrN9ef4={!g->o3 zW`Q(UzDS9j9$H>nK{QLp+US_FV>pY`kvD&R#*+P)2XN(+1B-0rnWgd6ns{J(+PGh5 z9zgR)l&N&9N^jdY(-IFHU&SdB^{QLlKVSFg{>uW=V}71$^TVnsBrTM-Kn|bo+3`e) zs2|Yk37gaGs6uxh9dt;L4I51_deyaQ5&2gYf@!D3@<73HCW&y$uv{dE9EOwHlef@f za$QP@V>&r&e%r1!ht34?xg7aJ`(;J?t19+iCajvYM{rvhSj*zf&5lvOA-PO#s^PDN zdoQ-G;K{p}ymAh;(1QIE3#+@2bf+n$sr#Q%HV)gX%!K~XXm3Ym+wEP9zak32^Cw{T=EP~rACs7CN{yxrk501zS9KO=tR`mQdmE#} zWy|eGT@?o(*Ih=I`DR)*0D#Zoz9Fg}Dt?Nb%AHr!AIRf0Q^7@|KqbQzi!7owN`d9TcF$eXvX4un zL$3M9()~ow%%HMguXl2Or*1_(TJpAoYWxuZ;QNX=Xb}KFpuj-?#7f`(#vLF4AYjmM zh)7__geXM(2*eDi^o-zWB%}fawqH>Q#8)SvZ@`~`0}bhUZT{2yAkJ~<1!etmr#Thg z+F}#DvizHpgbJrlyK|`OI$Wpa(3gW-{(PF$$Zb>(i!%-G?1k{+^hiz(=tP;2(=qwF zs1vAEj<@nCL5!_E_x@R0vy$?q;Ab@yMk|alGK!)mrsm7Z*3jpY6C=Zs0^s^Nfa(u6 zYK4aAQ>Y{`*8*40EoX%Z=+>aqlge1F2QCWd~PzJ$j~ZVSyjlfJ#2_@b+q(QOdj^~SLKNrr%Q znWSa0<-NZ08{MohTkE!;Bkk-@4jDd$A>;m$7clD8w(c9DVj|ezva@GWUO0E>M&5W^ z>!>iPyoayNN6YF7Jrs9U!x4or&)=!6JRAm!BC{}d_fe#^6H2&o1m@brTw;bN#BeS! zME@Wi+)W*+T|%uuB#gQg&ipXSw|ekeA*&!B#M%>3wzS7Boxqm%g$d`oR|@ z|K_D^bl5nu)n?U39RlW*99yHvp*iaqqI2fKTTuh-MAA{$uzxf#D(P!JFtnP0A5T&B z47!1$H$~dT^};!-a<5X{R}Ys=ZTRKoh<+^J#D|%=eOK5fkcuG1lO=ODrzn)yl0LPx zw8%PUT_JVel(cENdKuX8RJVWcX%kV6t71zR!yQOaqIn9QPCFXutW`B?IyN}&#Mc@X zoIprRaVeE|BY;hXz1vL0AJ^)~6iI4)Q;lbvj;EB+Sml!hYPf3MEDfa&Qnt1)tGmd4 zC{^2G|D}va>evRvi+1YjfMmOFDr=))*8!6{+=vi&db6U58T1xXxmQ$rG ze5n7hm$y(0)vwhJGe>_J(BaWhsbbT<1AJ?$Geu`l?7AGWNKgp3Y#JkY&YQ7~uSaINcCQK4o9m{{B}e!AR*t z5}j>e$Rmh-$h+9Xu&LxyPcUZWrBWi$OSm3#xm{QB)(X77ezM8n32pXin1uyXUGcFj zrwvH#+h{0r(@3}~iBR#b0Q1(&BQLPl@SE6A8Lxc1RI6%X@$j{#d^(3uz=ZV*mwZq{ zUV~Z4a`?olr8ViQv(2dWJ3Znj!0g(63ZSLb!7EUOzRNfKwTrBZuqAAuOcloJ4kYuFB;CLhj;DqzLlv{*Yh?chZZjF1U-Vk4iX&iF7PNMpGzt@ZUk9Yn;4odkYn1{jff%w z?*-SC6}_08;zY(P1HKa^2SV&%?i>w#6Oj7QyIOr0;041tXl@iS&4PyGCK)K9NfC$; zy<^5GGHgs^a?|`AUCQ*0v`m;=47Ma#?gF{Kv86r*bXMe{xftGRYbAu8;7MzBw?uaQ zL?uwbL$+|l9me~eAlOQ&Zwgc2^pNvJaD~7H{efbHiD2pnqks(BY*lZ8s&lwxI%V+A zy-bxxT>P;J)45LH!=Zed^vE)bmMmKsCW`(E-45KfUHt0UIN`*Ldp@|@&Sw2wNm~m} ztjzY-tHRaSOAyb8av~i^D;oq>lOK?HKow18*HKE%ioiV|UCN{tLq`{k05v_J3I-;K7JpS%C z5?^ZrnQF~M?k^Fp)0;M$?mR8GFLncAA#t~^=rxF3V988rL?hYYiyFbV*%4P26{J>j zrmf`&WSu4b2($?c?fKL?6aG$=%B;4+V2oNxnGC+K*GJn#_wpu;qw*tJ3OAj=Opc7o zyNyv;e&k-0969_Zx3Mv2Tf2pe3R+Muo|SkuI2@I| zfg+H}f-i6`;-O-;W?76IB3gSRDzZuyiem&8CYIXUk}#21(tXBhXn1HqP*dy1_EFu~ z{St97u;}_e-1OFL=0}s1 zq+{)N)2p`gi#0R)w}Cl=*BKj#9Ya=rxs_tCC!I`aFQaX-77@vEov=wvIPDzpq)&h- z_2N+yimiykKv-_@22~d&4{q`Dt4}g%CF8#3ukvWF(MI z%;uLl)45+BDn&ddBfkp>=Ph*95P6^Jj=*DlhMD!mF55Y%{mwP=WKp5(XuU4=3C;fE zw&BH#ulMCwG(huJsrU*8KmlODK;QloiG2kFh#&}r44_B^{K!P~i~QcL_1w@Y|&%K$4AZpUoyLWNedP#Q{?TY6@h`91#ycP6_>M1tP zkTKJNvE{v>nB2Y4COW@K;EFdtw^F*iTQ!n!&kSsXiYyuIvCc_Is|f@3q@C&y2?Ok1u)lPE!3IErX%A+6&o z&rclolqGSYtV-6dyYwo0ahCEAK9Vz{t(mT2mcqf?2{7t;M--JIMrCQ?hQ+vkW99U~ z#1(zljVJFL?HcKcSHEJ&$qfjofK*^s@NCCN8axyAu3PBg3yl%`ti=@7^o6v4a*>5P?+0*!T)s`pxr-SMWO7x&vsi=AnLLyHr04O(SVKF>bp* zskfkoi;mJ6wFWY35RI*MXe2($ygYsE=Q|4hg1nGwJvFZAwhlsBbznowP>{j}GUdh)2R&xG%eRFv*p7YsZ~y(xlE# z$V?-ePm7pmg_Dw^TBX0Pzbxz2W`=%5WtDy~5mL=3Fsmk)XJ8im?6BZewdXIAi*MwpJX{H@sA%%wHFxWWo!Afi=+p{r z>SLKXq_CHQRQM{mH$x&fIo13&ZFslEn$L1E&7#j4I**F-Fyqf>#%njl%_{QAy=66U z?xnvU)C;_)hy?7a;ODH%$yw9eSXyOu{ZI~e3zJmJ3# zo9bV~{)+X1LH>UX8;KBKsv^lk zvCV6`QT-{aV)Jav?&=YWB`6j@p}Gzp*ea;%l} zw_|(y(sVZ1B;g)CS&RLIZ#Zuu0t*Yp*1a%WsOR*O#X0OQq4r~83@;>^;&>DF#N%ir z^5|8~9pAs`y+iC!0%`hz2HkSn4V%+O@#CCC+UQJ^L_x%vu&z*BuPBYGqj-q5K)P1Q zeOkm>&uDQ941!%OOX4cLkxbOM%}Np3=h&4kEpqVI&7G;x&&J`N%F-f4Cy6Q+L&S}D z=?xPpw|OstLUJ$=oeuok^o^lc-rLUIV5pY2NC7JG6W~t9Lz+vi0nUa09=QyZ=iDP5 zU-QgR7N=Y3dg0TC7a|6NJ;M!~=p|S1j*T(x{~o?Df?I2@*}1?#y!V7q9~=Dj$yDUn zg~*RrYI}_6Sl4OsTIu8Gk<+Q5p}d7D&Orzd>@cIPHCD%(TRJ_)gF%J|8co(l3i)Xo z>OS0R-A=jr2qVv!2$LHrn3!3SPk`xB+H|v(F>$NJFHAnQJRwb%B0ml+-hy;KidA^x z2wsF^c-bWUn+L{D3PX2h5A(Y{4U)hl(PthKqrn?aVWS4cdm~h#FcM?qK%z@Y3PXw& z)rbf@G4TL27O8o^raZ=h0rh1XWI9Q#Y4ae}L-C<+-*g*Ur0Uq@2D;;@w2-+IE0P?q z3)&VW*Ii7HlVsK68)_KG79!L?0fQaNX^tU=t78<`tMMrIw9?I_p*LUut!Y%N zU@-MXs@W(uA_Rr$$5F~sQ?F#Rb^LEGJK8bmYbB9tcmam{TTvwKAKL4p>N0yqcYJ5uTgDSVTsw{*fC*2KiXi$;^8 zgx;Nx)O+Iz9`70T=n9ub4OPXHec$)Oev{J7IW_SX(j*a+qjuM`+HLsw#|F8+Qf-j0 zYCbsFH_*RE_*KwH1R+FXP(UCOFaSj+VB}Y{Bc>Phjm`OnVp!8d!el>f>leR$^>5w{ z#`l$XN7Sel1KnJUOcZ+({){_Uce#BR@1zb2j~361GqH$`pTo^H{R#a{RU2S=;lKdk8$fF@u8bdu*yzA-fG& zvq!vaUe9lX)jwoPCs)I?)d#O+@#|To`fPTaT%v19;=LE^(mK-LQt*5wTDCG&3gwbj z$el*Rt&IJ_qXUj*%zmMv%H_owbR6x-qX8grViu@tu0tMM$W)8oYc_!7OR=`u5Uf(= ztG8hzYI%{bzsH{)cOL%k7gqLsN1)MKTte;G{$UCKyYET1kSUcHxjt%hyX&?IC;qK4l)H`SP8x0D$i+2Nb2a=HcY!?eLupR3b_lwr zYpe#}bB4^CAQ5}OXG`p?A4Xk7tjTsg$AXlO(ge$DR#_y1L$4Yt=nBrkZy#i`kNYQ2 zTfV>J(*xBdFJ%X)Ze$-;lxrJ7Iu8;vFO_?{` z_NGQNFwZqEH$7`k`MmfOkQ{u}@Ovp;U}&!chC(;A0G=tKc{DQksJ69E-BT6G;5Tft zx^k&eXjkyDdcgKi!jm$lj~*RKaKYkZh)u|O%&cp5p$&VcRa+T`En7Q7weBo;?r2g1)gk`aReQXl{eht@NJ_HD2yB*O9FIH;En41 z=Fd2pj+@hV7@%LG;vN;znSE>5&24hQ1t8{m{ri=Uve#dSrZs>X*5-)ol2#J~tY9(i zC>UDcp*7}C=km!g)Q@WWo>$KQ>{8Hs5o&${$_wusGBmRhLl2y9RW}alzOt~1AYb~)z60p{uvS8&w+N5l~xeS(R+w#`z%wEp0 zX;4s!&w`2vwcKbM*)oRuRLZ^3a&_!&c+#!by)Uwt0+^RvOF&G{%JXP^9iyVuA9T3ai@{y{4#5t}Ym3Z1}FsMWX3d zGI!$bo{=$?)l8FneAuJ=5uP-Hts(9q2f>QYYtDo)Izi5(?4q~Nd)IOSYd)S}amV*o z^d##kh84}NQ_F63P~Al`)z8UJXXM=sp2(U9N7lOaV1nf+t*~=JD_1ziJR9ezV}vIS z>7PqPimy8)rjOh~+34S6WWKReyqLFXIrdyQZ{go>E^qQHni0;TpJ#2}k;xxS%k6=> zy7OE|i)Q-dlIbp!PkshjTBOumor?Zu-(=w17HhnEu(S!QaJy>zbk?omP#r%3bPrAT_FeO) zs3)F&{di;L$M34QOFZc6w-Mo+BrijuiHH+bn0|9>n{YroT6HEgE5D6Is6b$(7zV_~#p6M$D`t9rgmCN}EzcSls2By~BcDh%_f+21v@TWg(BDG0;71vS2%!OibJgCdPppqembLr`2+o zczy!B9;HlU#q1G$;mW6+xlz9T>@cC~*88FPa}uXx^Q(Rzhdv)(VS9MPI4rpljm8lk zlRB_>6kXOXIjYt-qw)&7PPv*tqP$+VsgMk{3g!Wh>YgfXMkaqluBjdIi}#TcPY1k= z(e{32ligwisBXhvP$)Ggsm1P4DeQ*m^I z*5AeHnx+^9tJvlTr3=h(rExb5=YnoHXQf8M8SFTIGnJ>hdi_Rb5z}`bn?xxXn(Qzi z4|ShHt5mTPdI@iAqc|@D_Ha>^?JG*}uuSP?3E(Tf;wk)?S4ur?}uMuFokg;Br8HM*2K@ zlsv|leU}#)m@t|oLW=z>haRf}aE1!YdLNPzL<_oLGxMn)d`~(WpjL3=+gz*G2L(Lk zBoy9bFGS5jeEd|P9-+5WA)6?&km5-7cWyJwmsq^Ks-L(k3h*l?Nt_3CYlE>WlAeNH zrMIoW-_cZsI5u?+#S<6v;XzbK-cvDq{g@Dtjj2iyb zuL`v{`B5zRr)^x@Dnm2N9C1#?&VfJELs$!Fg*TwYC`Xm3F^^kyH)vIs_J{L!H?|io z0Fb@glZYw`kD|>4!hz=tKoo?-PxW=G-s`LlV}?4DAB5`9_UnORUb96BN=F9MXL*AG zFM>3XbP+~X6)Aojt0T~MciF$c@;THmq5%#;&Z*X3Av%_D;|N#2cQLx2I6q~eZ9S(V zV3{$1MMqd4gpsCDMEAQMM~8GEOBJxCV&?upIysHsz`AnGi+&fSxEV^pIs$-ozk>6O>chLrEVS><&ryeb-HRSFc!iZE^{RVOk1|!Ai(+9i*r`J7~ zP7>(U$k;pt)U%Z`;Cn}!MnltS!_bT6$Khktikp4Nk+^}14MCgyCqczwl9lOd!BOBS zt~2*h0UUh1!3oMZ5hHoLe!p#i--&9K1xFA_!a4nkNxp_MGC1@Ex@d)-5>~QMvprqX zm$}_dp!l9xT}3yK34*}(2P*5aZO4S`NFAg!b$0Z;Lu7a$Cf&EP@nmQ`jFJ4(A0a-a zO+X4Xc9J6_FT$|wB}`Ow?nX-Eg}*-mk=yhYFm8P0wQx10UZr40f+ z5!7n3|f0$VnL^OY(lANRLt zatbC({4#hIcFZ7y(K%s5xJL?U5{!I1;7V5@JdT z&#zga=tEGqPi~l|_nd{7wt{qQEi_Fd(s>!{Q=-XRjw%7q7c5$FQ9%SlMrW91u?3hS zLrSL$3J9x-5;Id{ah<}}d~1+U%@nAg1pp##KzrF?q~*5mqkIjGQ?k6ZK~ioexT6DqQ;-R;{g0@PnAUo?H0)D)P4yJ$mnU~V z$tqyt{cS1MX4Wy#okeC9bZa$jeoL=7iW~!Xycl#(!t=Rh#UkXEV(WWgSaqUcTN7t9 z{odn>EX2BU_yiOXpzWSX^Ls&hUg_ezV8uRE*t5v87QiQXhfhT8o*)&m<+ z&dfjgihFt?%18I;s}4k52sihhCQ75I&;{i#p)&sO+@cP+o+-R~p+9fNwIc3khw)Qn zS<;JFks;FL%wJ5WEq`VQ{n8#IUP>?$E_5Z>y#pnsIPIlDC`{V|0q@+eF3RCm5Z4BY zQ%gWy(?>O$lCb}|<)m;EH?ht5 z1Ge@X0jrXmnRR&Bapqt({|zp*w?RsCQ!CS=^7M4cE&FsKW-F;?RgGpceTp~YDBu1* zrWG)-S$0X^$a%&UW=8_WRFKjiQ|7E1+nB)-`;L&w;j}Lrvri7Vy>Qu`))A#HFs&lu z>5KDlo*X{4H^ z1aHXap<{=b+>c(PUBUidEMh>aw{JjSpA#hQf8g_PJkTH6bXC)2SkMp|C}cZp*pl<3 zujY9?UVN1#(mWz_*o1h;Px0-YYo@U)Uad?RM^@VJd!rr7XxvbvN@~xeO_EgUP0iBH zsZ!K+LN*fcO2p6xzx)K4^p<_=lmI2v<}~qOkuX(G>_QY8uek4)w&X{}!Tw38tFr^Q zjg^4$5Us$TD!A@xb^ydOq3;G>C2#4C_bjLoNdm@9uznos zC%PxCD7BQDWlv12+J%pTGc$9uRjp;BwR9)-!A|qio8wsy^!J*^ThB%YReI|E(b+y) zI@Lb&6Pk%A3a|S2_HILn>SPCDUXa1ME@s>5cU_%=j;2;sO#PdfQ)SUS#RJ!yO`?FYR>{_ZF{BNXPgtK$SJJ-mIfSD?BYi1}SrPw~iTKdbb zE1$~No!etJu{g3-I6MMu?Lj`TzL%;UzNu-yoo}W)edq{~J>e?kN>tGP^g(Uy4#&}C>U}H~l>$iU=aI8lW$e>jJ-_8e8lnw#3=imO7B27rHS8yaz{|lcz zC?t&cH2&rKWTk(I8}V2|{FVCPrVGyjA7Fz1+qqI4W|y|8yZtZp-SW#I1v-I*`4`q; zf~OSjhDQ8_BG*qu!VDjT|4wotI73`8c0z0KuQcG8f*GR=f%#u)(n_(4P+)o<$zN$2 zF)&&#Z zk}TkVjxGLkYx+N_7J>l3f`Ko+U5#z*RnC8^qyHR4B79Ynjm(Bpf7iTSsS#wZ$+ouv zFPojLb?g$}=;0}-N?AcZ#>^5N8tZEDv^vwjEU6Xs2FGK_@^@%r#e-i%k~Inc8m?R!)Q;~@ zW+vG{kR?r9_F}v?h=}2iO10h3f)%@e3I^Q~Cb=0Fe5p%~QJk#RjRT=Y^FXdVuUq2N zp}A+Ix?uc8#_{BZJfx4eWRW}{Q{&czBz32)>#jF=?Llut;Aw*9DNCF5+lxFP|3DnM zi!{{PZ2PwmA?r#XPZMs+AJ&vNAwTs6n{wPJB&9kg9RD-w)>>#fZGKi3qN_Z0!`Bz)&~Js}QLrzeqk>PU^)DPA~Xs2iS70E967 znGZp|%2mY;LmjUd?s`-*Uy>(W}4nDW0 zQv|_0oxtWCG4WGWVlOg7q*>h%rr8JbRR1jAms55noF_qJ7cw3}uX`FkCE+9_j1XD2 z+Yd`kbwW=9v+$zwcX#& z^1j?frhQLu0*#|4o8)pq47OsL%aYgq1bm5CGbf{OU4o@#3CGrwlD8~lH-u{7)P?P? zwoMSP-n*hkqD%Uu!r7XRYPD}LmxTy$qN0{KyEBJnd#=XbyP{`_xrb;gxGcK^oRyhZ z@w%NobIQqb~J^J^}Z_ z`48{bUE*AXgGc`FBDdZ%>tnBbb#R{m=j^^ufR(J{$Eu2Ds$16ZCqTR@q=FUs(rh*1 zqb92?KmLt?TaaC>)$x@NHJntVF>}e-wu!pE_GCJu;V1hTU6(J*)1NPep0&!Dmg2Np|nDp}(w( zx;KaV1RZVG@*ePmC-3WJy4pgg%J94VN30P@2m7Tz0VRc`iBXf7T+SxhV?RydZ@)cYM|Yv=NGXaXL6<#|zm?)z@f{P*^EgG= zrMRsw`H``&Ubnz*mpid-#vie1ckH98pc(Ct8JLQ}TxVYFKYaqiF9vsT8dbs9 zytX`>(>chTUy~tu&~VJuNTSg4{(J)H`EOQ_qj#dOXk->9AIGz@*QpUMF&U4Jy;(m2 zaGtBy)o(n{OSNeTUgsF6$yX0}#)RSE(Q&vQO=(o7ulo;}!ykmLeZ--`kKh*^wt8|` z-v4uXfG>adbp_);XSM&hJcO@PS_K0FdOP15+v)57xV(Q3Y5#F~;?$dii%|r8JV3P{ zmED(-Q$$P71eG3>dEGad(JoVTYU*~07r4&=mX6{nhc4A)e613L&aAiv4$>%(2ucB7*bUFBdVRePu|EQLv^EHNp zg2u)m$^UA~*@hfW)0-+%Lg$8`9|4V~%05nmCYGb_h){~*nkUA7!em2Trj{2{2$bNx zHQp1UN0#Y9Hkg-qIX#^WoV?<34-qM7l^05@%#(orDQ%9+m?bt%e>Eef=_$l3*LIb4 zuXd^huFCmM76n7fje$|x>4nFGc@|T*n*-JwuVA+K(Bupm(qW~~O+u@?`YJppEfn(jQWXoi z%I4OlD@5ZH@N$pY2=ATGXVQL5W@gg^%jz!Q#qLToWtSe^(CPW6WA)%Xy>vv<#<1A; zLU)MqQjNvWcskZl(+DTalXqrCjequ3QpiiKzBe%uV=Tl*_bb^SeJ)5`@7}|V=i!le zf3K+`pM=S8F~Da8y78T$EmK3UEPyjh=z1h?{hZp3e$ZMuOVObb|6l;kzZS?qS~ex6Fo*dVaX;+%(8e>%x8oV1c#o}`C*R&-1$R-a_p$%FrR)Z zuD<3BbD?L(U8tf38+sKV!^RH?{*PJsr8Vj&z<=kwD1y=mgmjb(=nCx5ZI7p)8@AJe zJO=PfCoM$MPjs}vK9dVT>Ub_DeDkJnDlIRsH2!v83^Ex={&J`K;H5*AE+ESF1fWIt z%W&i4gH7mRn02Bp{9T;=iA*qdec_KRjkwJ==RN$17ppQ1<1nk#oUYp^Kviq-m~@6O zIUuD`eLfqnZ`WKm>JuOlXA;Nr{#xrVw;JTF8jgW;1v{T274Z`2Dc5)GPT&T<2g$vk zq@EXV?(#uqlDgF)U~VH5l36HL<G;8JoC8vjJmVT9n?_A|&^HgtS<_8;0@2LL> z*%E!0?A?mpEB1kqjtPcISA08CeOz?+svktLInsS=MgDR6^zG)`DY0~X3dMUq4{J== z(5qL=<|=i};F_s~{nv%eCpVAFh)g!6!AA@s>ldpb ziLs?a(@W|z7})*UqGsl$^fASU;d`Mt>&fHkI4G-lXvfZFn+Dk1Q$_@N9B2)rCN=Su z8gN3CGSmR=(&2L7s9tq*G^n;IGulZCVQhshQk)sUXTyzwn@Bmt=E7DC$iAj`aMdTY zgnL7D%sWhekOcQ{Xv8z}*mBrIYFq+Jy-&CPKz66y#8+j^PCbw)hVB&!X8}cFd4#tyT%Y=W{z2 zz=B!b3}(uGi=EGFMA*&#jr|w5F=sX_!jA>g>h*C$%547{NhWw*`|*5)R64yQ*&0b` zN)=9@bzVMe9Z5h)x>7AbL+>e3q(aNqM;+XWI=yu{7AL*bgxcW z+*zwrLj{g-c7x?L2;^POy~lx=aP?8KqE7+Y=E#mQ^F=#2^49)>1=uMAac0KvL8Qa^ z^5TIL_f@X>V%_Od@Kd`@5I+zf4Y6x3AwK~!LPuHxU9N$QJstM)0v#PT2z20X!3xxy zK9%+yGDvhD!Q-Yg3+iyoNUsrd>=Z5f&XKXVG0ngV>%UwvUR1sUZfuoqaWgPbRl=AS zV_SsEP}#_Q^b>m_ggAGfo6ck!=)^)nG~n*5dfXb8JSn6~8DV&(i1v4Nr2OkiAlQlo zdXG;)6+5YTgFlz|r5lv|WUx5{YShjEOr@VuG$!x4WBC=I>tMw1X{re?GMM(LEt7u9 zT3OR&XCh}dGwvOTT;<7U3B2fZs|ad-%2m3S-dYwff7a^Gt8yXr)Eg4K!|Ak#dG1~d zk~vlI946XcR-7zx`gx3EcLd&<3Z!7z{X3XqF!c(tT{z%aW67qeK}{_opGa5G*VI#7 z3P?LrM7)w6T{DCb__*YU;6vJgFu~m}t-d|N@y%5*(jLVURqH_w7N78~_GUa(B0i9i zz?^XW08blSB2XddG}))nNT;Ax1i?g6Bz+i5SBq^#hJwy0A*e9$X;eJ5%iq-dzAbHH zLt}w`>PT7C__mhm1}TC$S5iJNn7J=96T6(G36PSv67U5!N4=kb`Z0sKj1zKahGk19 z7|FWvJ(Zy?95x2n*T*y#d3?KZB5*dpaY%P0xM^qwg00ac0as}8%hVB9t4Irs)@ZCX z7lxSc&jS!JkT|Y`NI)KEwhmE|cnO&u4j4q(#iW;UP`|s>dsTsZ`f39oYZ?4}tU^+f zy4?m1=8_@Ee$srO6Y&EDghKuh^Tzwt9u(vUlB?z75R9PjmDBv3rBP)nAjrGs4MudE z9S+*a5Elr!56O0CAc8D7!obWQoM?+hABhWj6p8a?ih$}r563KY`Cm@|4NQ$5jAyHgF+cTt_ zO-h>#`@`#r69a;PtAf`Ob?gZ=6R7B^e^(9a)rZkEz>1gOK*7;i08*5FSzCw$=hQTp zrm*>5DJq*h0>w-UiXAo(@W&J9`~3JhsTb80d#Qey<|bMA+aQ=rJAcfSrFT7>pQ1G= zua>8AvcHeQ5m-kvFsI>yK;unS3f{2{P>(o+wp>49rR(Ajl-kkTb^ z0_CIvq?pD6e1xM3&c6Q4-1*2j#WgZdxSo&m3KGqa=PcLeSymr%A+k@v?oqmV0Cg~H z1_XE!vQ?9bHTOG%2dcl-Nk^_Xum8ux^CY|oXt1i;V|F)~^rfS@f;3IbwPjG{aj zmu>@06Wim5c3C`;{8V|i2xT!)VV|kIYT^pAEJ8mV#89y2N_R?d<~{6ptAF6ob)`25 zP#5y=MN?0L>S&6TJAV`XJsjRlQY#lhS*!|NoJwS9goY+~?*~%4+8t9A*6&N4UIR&< z&L7KdI6{Ub{Bn!@+z((%r5C>-@eg_7%32jz7A_E~aqBK3;EpYHgS%1zogPK;vPaWk zgxFE^T6Tm9FX=W%BjUXQb|x7;B_ohMRi_wxO6bn_LuM$)5@+Gr(MKGjPMF<{@BzzW zFDtzrl*C}`Flb2|B@i!k8$`mwn8@qYCiM;N%>kKujM$)%RF0!b?mH6_qm*i~Hq z-si_}tuBnd4A^kBkfO0W2Wfpb-2EWGPrwhq=MgBZ%vumI{SiyA`!si)QFXDay&MR< z3BzjO@jh?VC{dKm2u*%*)(#GF##CRQ27@r@xJnXIP?y@18mTebVL>tdR22+G99Cbo zSS{KUjI6^iN82hkDKaRy7~xlpX-b90UJtRNSTx87DglfD#X`ira}u`Wk8~X1GAXc4 zCWu$QK|%!kyBrxIdM5D-zxACYFA}b#qx)fqk>4Zub#Q~c$PcN5J(Y;DfSwjjG2F`@ zNu3}G;uJ5|QoR0c&tK#_<5T)}S&qw$1>-?1XjT#($^jlA1SA5~)mA6jpOizNK#CrZ zB_>rvtX%B-g2g1Iwi7}vFYyrHZzT>fWNOf`pk)!O8vg2B1U(!h+$4ODV66!0xJxj& z7^WI8aA1fRfj|vN`J7*R?0rXwu8c*KBz?flzH;o9aqRX20fE**VJO3p9&(4sOrX__ zc!;pg(tx8pLQ%6^Vx*K@uE!XK7KhyhsR0RrgBmiPKhVR$M1;H0rL~d6djk$!jCGj# zc0Gscjklc0lal#C-!}{3jZ}fTaMoeDay_egqL7)5W1=MX1_N?H7p7brMXl8QOW!vT z!VxeQ-};hQqVNJB!ozw9G*PZ2nYtI*mq8#Znz3!qJVFE8lf?NZae8>MhlP{gj7ZtH z;cV?Ge%LNe*ll-fl}wq<8Vp3len;9PLJP$TNESp3pDFK^rZ;yZ=-b(e;%N>P!1cyB zum19N{uO*)*NC|6G&Ioh=0fczsSoN|8Z~v5#P`Dyh62<9(2n*-P#8cDhUgHb23%iX zjFkZ;?umw#?U1BUPL4BC&|-d)?4WSHDa=&2i1MQqBuL;vv)Invi15{W0DA7EYO z#3qAyBLg20g_oUV#afL1SNBkY*&NFUbWnZ>5dY(urP6bsx1FT_C*}tk`0IAX{_rnqL##-u zRGl$T-ykHgd0WY0OPGPIVrd&MHUwk|69V0M!`ShMkr4VV(ohtXVp5>QF}R+fV{F0a zA&QBRr9Kj4!_7d0?S@$CYL%`Mmb09@A0N-s&;I~p zX3{C~*BqXl z(GhT=1r%AFLlqxqoDq^*L%d-VLeU@Z5+wqponKz?z$He;&hmcg&zwza z8b`OrO0im=!~K0J9UMcU_(na9Oa%%L7@)~>FGj$f=cN+V)p_D|tu>Dbm@K)Ro+;Jq z0YD=I_x5shB$96W!)U@qYa!KBxL`!vtcRK??$-Or(I8gm3e^%)XDYO^(-9~mS3a=3 zMdj8|Mrf}uqX9C51^)n7N7A~=w8gSRW6YD`oPa#K%EB;mI>0!26p@k2h_dh4f5%xN zq{40QAevf3=e#1%D~wkcEKT)>+6;$Sk~;*y(T(t_SdOVTv5y%fYiEYMe^_Z{Mn#G9 zkpP2&W8)e~UyyT=0)*T&LDZx8`eo#~x?|=+lFk&HOI7*JX?ToCLzBed#x&{U1z^RP zNlZYC!5;qrK7U5HcqDw})a0hzaf?yjg7BD#K@(#P3TTGS@qj>rd>@`LxUDI3{;(1W z8X3qA9i3syl})&7fX;&&jEOK4#wN#^;lLyjBu}INnvhqV7L;+$SQtk7oR=VLPmB<$QrdUEOa^F{j-w4+79Y;?yGd2JKb#u5DlgOC zI4I2tRpv1S9ul?hpLj!*P{FpHR>;T%LO`*^WK$tgSd}B;EAx?THva%0K?C(ayq)XL z4Sq`=6R%#Eh6kgx3&!^jLW*%3>c4KW8A$LG)(I9Hq*#WLgz7Pb1(yKgItNsVgjLf> z+k%P_Y$CB5xZqX+6AebkAhLHHQGr8_@R*JgH}Cqv;ALu_#20C}g_y0YK&rg&=e^;Z zB%s4f^5L~-LmHpw8A`7iH#V}k+JZ+UAMukgSR~RvISjLaB~K(5`{;@?G)5GD0kIz$ zO4{x(m;HGE0M*tQ06SL<296ecckk@yBo8hrj^4WCH-c$x>i+<*Rfz+WKOw~5?6}gj zo6!BT+#nDgGz!TC7dXA#wktH83KURK#ZNKkM)qW;zh&o2D!mY)*fR|@5V{J&>jXa z5I_;99wq?kn^6(t0Am&xvYkwQ@aRYp3|W2QS}dUoupFDcYGNR$CeVw6*p{Z~^P4H4 zN?SYROVV?X7HBWu{jc5ydzHxl0OtC89Q13KcD#jeE)N(KJHYSlul0)&G|`{;cj{aq z3g&t8WixLPIm^%(WM4C<`Njk?`Z>!l4Z){5@V%`btOsD^i6e&-&v+sTu^Rkl$w@Z- zryg)*qlxR|`pL|MhJIh?wT>AiD_$6Q(CZWOQ9_7_H6nAlb(W105da~ypZAMG7@}Z+ zaL8d-Ydqwr05~HK6AotwYJ_hkw7zZ>>9RP9MC7LsS3+ssz4g3fa3W9&v`WOWM9#1q zLJD+(N9J)gF|7nF9YR{>SiPDFz#=Bc0*O+oek1kAXp>QNx%A&t{{WFdL#9bSC)8v? z5b`hQ`oarF&_w%r$TQIBTDvq}q*$Pqwb1_n zStg7L%nlH4F1f~a$bwYT!#u|^IXeiYVBgDkjngQ=yWwL^{XpkICglGBXkrLD5J~Km z^88~?G=fE{`w)CR;=ThC+{{FL{{YOLjeap>k;V_TkU<*#pV`KH(=#^wJ^h|U{nuZIBsnwu;2p$#eefGnX@|e#95_=zC`+wN z-?lPM#tllG7Y3kEG>i8Hc#|zNnC_GAe-;5v;UnIR0L83I{-$Fxc-B m$eFN=KqEmelg>=pO@f%f(L0g|jwJk9H>~#I{{VmZfB)H;t0nX6Wg|JCllMYlP9*FC$?>KqRE~2ci(&Ket&*_PMuSA zc6ImaQ`OyT?_T?B^=lV^`cpz$0ssaE0D%3w0ACvbQ2-@%!f5XSFxc!Rj;V&CNx$4KUGA|XfcUp z|JwjQ)t0tiA_ID7R3Q}E35I86qN(Rl4eBL9kkyUq`wv3mq(mT1 z-@WpGYyNAIuQsQf{0RtXwQ4Cv3y&kUfl9>-YuB}lyQp7|y1L?UQ7+aICR<)yj`a`8 z|4-+Ce*8Z}I9=>5-yRz#{|=^sO=5^>5*7evh8rV?Us@BQ1ah;kcZ6Wx?FH(_HbENZWQ9BvE1@Cj&U0W06 zo0EKz`lMpNCGGw65K}@K#$}BN!xG>6z)8n1fN%ktL=}FgjY`dLS)WEY4W!jd(8ohn z1z9>$WT9e3>v*&Ns!!N64Z2VIU6GMy3tX4%6WL)(opEmPYMP?pH?^wTF6BEIeysL=y75JJP|M>u$w_;{an8xpk7A}Q~p``?~?Aliu750eWJqd zDD>BCYd72YlI%&6u}81CX3#TDzb{Qw)#B5EN)q`bQVSg5Hi4d7v#T~G?DdebWGUrL z-6Bz+2nhjk4bvYwd8AahW;&?ZwZ1RU0G2 zWrHKASLoSOkHaR&k@=Fh894n?XQN4Z$C!j)jBg=y;KsOIoh9M90le{gq>5nx(r4{f zcKK2|8EDIU(YOHs4SINUo+z=GptmmoFUa1v?D$O$hVQTbvyV4MTw(d604NG4Tv#)y zHZxZK>LGz(saj=jK7=e3Pi|yDlGPx^4dec;Rl?n$OfS2HfOlF`5O#u7Y_1eJY6LAD z7TcwaPv>0XsADXLyk^h7%S&L}so?^oSW@x5kXx)>w4BAFBg4kp@|C?OPFl=`%?bz; zZ#9hdQ^zQ^*`n^w{!+;7WsjX!vW!>$+WCTZLNx_n^4^YQrr4JW71K+td?PkyBlj5H zx5_?fy1$o+H+3EStb|ddGi=&Ow22>rI5x4P3jD2yS(9LYv*0b;sDU&?fUNJX%GV~w z`MyK#N-lIv*%X`TRhHA(Vd*T)UbSJ-5`Vl2POqr}GQjn${AI8Gcq9piLw(%~IHL4H zr~CkQ24PsKJkCZrn6+}|&%VP_^M_>XPt2Wln_qT1P^wb1c+aGdnyCtCbD^ggJjzTVm?X;-F=RD6(C^Ql`eM2Yn)_q-^GFWkkjo3b2O)GH9n*3r!K?P+<*dmvcb-3viy_mKP!_hjUsMTy>({i z)m%h+wMn`Xx>?~&U2R)8JDV0zJ*#2RT2m>iGOG-D!D3Zt+0tD21>oD9P?lw#Sf@|4 z`c~qVz2Wt{mYCDjAao5Duvj(n@O5}~hJY)!)h|KjZ6__yUO+er;U2TMxiWL0+;Xr#1S zq9Ld0;pex*>q^E9Hg-@!x81b?TYJWW=*30MWI?kV0;6-dOb$&W6CMV2!u}@Dvvp=9 z?h2P_Jt^UKDDr?^GNriv5+PYxM zDKmjq8tt1R#VR6(&sZnb8v0AtERglQ!TF4<0jWb^jo>s^2Au62brL9nKfDow!rY(= zXIpD(TH=PHYA;}WGV5z+amF(gPw}I+4)M6lE;w_oJBq|>=v=WUj@{QQnLQb(WZhal z4n?{_*;M;$xm|}_xWssW$?#;+=G?jy+!eb@3dzm@o_c5v1Q?u$qDm)5LlYldY87y4 zPbQH;k6dFVgQQ&Fw$2{zv`UX(Up04kM0l_d}Vuqzo=T?L*wPg;}BrE#P}*tZY?<5cFv zR4Zm%%V15_7gkIw-UOZzY)TYRFG>>Wrz%1UA>j)CGpWtd6(tT@DowSF^syrw49Cz% z>+(IjDSmmo+E>@91`=o+o*pM}gc~A~zd7SLXP;|x-p`+I9?c)xnA@W8q@ql*H1ccH z4*g%2#5~iMG23mE8{$YEi7yHcjCbbN!*j>#z}%6@hmrbBfSXL;je{vA?6fA z(V^F(!ZCZ6sEIfAWTKH%Lb}~F zql**oLpZ|j4f|1t5|urUy-hBdId1){;bpVT)RwG8pB103Js_M}rb2QsLKjD|xL$I@^W87^g z6l@jr@_n)n7|fXT`E;|=IHt!j>d({BsT_G3I(;>_ zNn_|G&vO?sO}+W!-L3n52~pA^eJV#R4$|iGrAG~`2t8F@8ekk{l*K*lpHyJHwIL_I zJ)u|iWsOezn68};hKhTda(lckZU~3sMFCa_(XVAHj21?xx;@^S&LkDvk3-_2ccjDG z_PTK$@u#Rn`?p?k7j16na-OYTYgXcuwki|?REAtigW${Gqq9~nL z4aZLd-R@ewK5)=kla+}PE`=LOtjJ7ZYfO3KP%1GyV>r$NepVy%5V74!+pH_x*?G0- zBtynGoDbd}IH{~0(aRwH9Q68lyD%fYIBjM>FV77q&nC-L_vB``nE8>Ksv2Dvv^#l* z8`Z+NndY1BQw)-vK|+!aR&v{XPlndE^Us$9|KD!Wzb7OB z3Jn?^1Cxv#m6Z(!i-Lue5?h4o`ww=`e?AiIKhFvb7VK~TbN1ECK3eIjW+WB2x+592 zA;i?nJ>|ATcgfcj)`5{9GDt^aZcw%{kRl(tWiQcIa=F(h9@K_sj)5Hx4vu}Y$mkPlTgtb z6utSF0y0ks{PZy=*_bCO^J`c1x7S2iQCDF)#pd%>f4d#PYj4ZA_sFn5x@Wimq|<2J zOUTtyM5ZAI^PVK+a&RFJit2&nucuqhu#m(0`Qi7S{dLg~SkJE$e>3}bkeuL+F63RE z=7!?u^Ak0(Vd(>XwRjSlhF@`!FE;XMpBbj*57EvNl<=5cgN5X%O&=L^jcEyM*CV_# zyDPQ`^Q3lsX<=F2%d34`&gh@0A-o#P5Om(5F}?s*Z}#)s&M}~cq}Y>#f)@S{HPH+- zP!hD9)rjI#&P1OWO-r-pa*8`^Ob6>n)F`#!F*U#E$qo&sZVVo%>(Y%6o?AwO+_f*Z zB#xBPF=2x-@f?!*aIDq0>a+B$P#6eq4(8*ZCS##c?&heEI|MLfY*HZPkX$z#NfNImr&C5@O(iX782Vi7fHp#xg z{6LE(-_=}6v-!}Ql#*B}@Dfc#dW#tA{yfI#lJl=>{OG2C@edgsJ?9ok=iZyY0_Kud zy$17O27|=AWI&z%h)cKWR-Mfo{nuU0Ozi#N&g-A`HrAnhEiq=D5zW~txC zV~Q3(sbb}qn^ao`^Y%u*Po0>#89598??{=;={_HW{9BVUryTBHv9q%%vmT^a$uu4FWDUrxsHn?xl6uG6zR8~<48uS z&n8>uW!{8G?8-S4?tRPkWZ5kvk|5q?>j|@XChpbSO6RMzgMtVLz+FoAwc>a5Bs8;H zlPTxI&EVi2%wUeV-95b<_-R7mmiRQT^N`#gBv1QgD{8scc#b=<^Qs0j8YZZHPT!0K zR4z;}*tlz3&=cLNn_*EjI);5FWBuGNSfUS?qpg(BDm0Xj>QpJj8bgd8K_9zaVV%Zw zj)t9-Nq9wC7JY@*Lg1U0Z%H$Tb5wPmL?675+GVPE?7Qg0478?QZaLMdD``Hj+sX=* zvt~muNd8M%`Y=KtweFkXEVXWlJx%iv4K<4PTYtjA`o8ZAK*UKa(Wn*CY~4cTY3i5N zl9^+rs7fyvd$^2mD3lmjauU&PYKC)!dA7~SrfQ^9FV!JJ8PnVGzr>q*fb}bQu{d=~-*`^y( z-j;~QeB`$13;naKFu_ya7;Sa`kCC~S-Z&%F4ndR{AZUJ!p92>T7{5?%s=qAv%c4IZXBjM(}l@eF>5!izQsXfMr_Ssv$}#N9dlRl;jk zs3bt{yt+QS(cV8{cB#H=DV~B2Qnphm3OW@xku0k9wUgZfC|TmF!0`h5Y^9#QPv^MnKS+BJ>8&!f~_T6nK5;T;3Z4@mqHII2Fna ze>1vxEOXr2Rm)!y$VTKB3mzZoigNa7!z5$wKKrv|D~+>2r<5f)u14v~VcMl;BC$An zwn2FVeApjno;}WIu9p4+ho%}uQ=a!c)xTS-Nkyf!xRXziJieBa4;-=n@kbH??vnf?oaw`!M1v`@bs zlNa2Y`=linD`;+p+fO4NWd$v1w^RbDyzDF6W)04CDLu?1n=yS; z>}{5L5V{MYz5J%gQ_$p$L+T;4d1Q*Q$w1KEbf!|qiE#kGG-)F0Z}UKah^NTR%tfmj zhrh%3i7>^PJYx+EF$D#ct7{D^E;#64UR7DScHMqk2KSSSf<{c7$f6!?U_Nm%;qEX` z>x|4c?U~D7za-vIn2|d_^xi02{{1e$+^TL>T!rL@o%N_&;WB)_fL^EgOFIR2cnJMD zbNNg`(yyeRh_lAkU3XR@WtjU0_y7w1ixz29ygRDC1a?;N&HvZNJ&BpvT@GV}-C$eZ zhIRS4(m;}J79M;kkOs_|i=mRQ6n|OiXGiLednnp$$F%p2d|@m0^u?!5vaGl@22J2l zv^y)Q-s^+PG!!tFr*=NO?Rwh@G=H5tCu)adC(+Is=AB!)V*wmfbUT`R*bItM7&Pj! zX8f7N|FNml1U0l#cXod}am__76?OC1!4z%C@sXs*FHiM;hkH6z;KnuEGF`dB;u{cn zoi0&NKsp)TnIIJ-nkU39H{qEMN z+NC#L&w~An`6^TIwJCPnaeMQ4p`6ab8m4^piRz`&eo5pBPFk}Fwjewt? zzRGly^oTFCYg>dsKQ9}67S%3D*wGO&4wqv3M}yZh`_;QO1$BZ1ekZkGwuyvf*U8m$ z5YEm*5r>!8f>Yrq6<=VqC3BFcp;r}1z-5#JD{C^`*w|0B)B>qHvepU(EwM+Pw`mEq zKHG3M;XHrd#GzQYsb)Q$ca~*o>FdpWd1|dpQ$PQGk_ag?Hjp%Ww9QKZOGVb?a9(r!CD$`zc~<{@-90Je>+i$%!53-M|g z$wXrgrK;#i8RMYElJsbhcn*B@B;d_D*XXOApV+^}xGQZZ1EcuyU8{;2D!u??ufJ?> zJXS1jB9x!)fE0{Q>a2`(V95Yw&Dphxe z4uWkJxtAUZb(B7~+2yPdEaK9EGxk?s06$H6Qfjh`^h24tDoi?ge7nT6$2*&2I}+z= zL_c+yHj%`+E5a%PY?N)bvc2fISU#Q;X8&!RoijW<6;P@&l)x8z7`_D<0WmdLVSIo%H zP0wN3p~L#W0@iqJBow2*uak{K)Bi!WgD0N7DNko=?vMx?#O`EFsZW*9tX|F7%4}9% z1g7i+1TMZ=0Y)CzGM|>bD;qP0Fva9miH)%|WUe_4{I&%=d>peO_a~Dw`u>Vd>&MEh zH&XO&d0>pNZ?kPmnoPU?WAg8X{m3OwYTiLB(AOGr0+l^)wv6^{ZJg~<6o z2W1(EE7Z`9#X_30one$!rgk*jGLa)3IMX)IIl<%zmU)M54n&=wRw6W$YiU&MG*G4p)4}IvbSa~M1o@m-BpMagb*bZ?Y~oi*G;0%xbA%43 zpe$a9Ex98mYEPm?jav!qNX@C{k4;}wuK4A9OA{iwco|IG7FSmux7yp9>L{oE4s=;5 z__8h-xcKF|jBR}Hm*PUEo~-i&is&Eq-F(t>$Aa&^r=RDNpKnbaQ5tft9hmdt7_q;g zjjzVCz3_6i59FHjKlJjSa8j|?O9*#Vn$_yiX;Z4I6@yx6-G~XYbFH?BO&YY0e;$v; z1{f-dM4$x7lVy7wF*-(ee*rWuh<4EGHgug#n|?eSuJ_L|g>TrrsfT#QAi7dYI{gG) zbk-fL%qUnd`j38BtP`UX^+ULI8;o^&9t;_AWe{Ghg#=q~&%UA!60qM#@I!P)|6K;&^?cB%Wewc}0~|Kk#9RlN=jsHVoBbyMAU1IJU6<;o(> zL_1P&W8NsqJOn!5X~Ele%iHfHYTQTzpf4R6 zdgz{wPVJ`EL-Tda{FR0y$X(V4iD$P+7$@n2=Wd(lo?Tu2Eg!-$Hgdealp==GD46g5 zqe?wIY!&poY$S_9p$n?mYXFr#=f;u3owvwm!T>fn&jaG%x|Wi6QvTo`=?L3LTpNGk z&&n#z->S`8s_Pk4!d10{3hUlon06Jjnet!x86ZLHpte?LQ`F=jpeBIYNln;sL&L@f z9)8VaG-{mvE*@?u?Fi%LZ$n

3k|*hom_})fu*z#bvD{qYhbExd9j?F&%X)_h{F# zxLUBZix~t1_ZZ<`U9K@qJmKR-3tQ`XQ>J3I*9BLvoN%&&{ucmj8!kPF?_0U?P zZuw!{JIp4qX?(K=t@E+2oxu89-tol4o2R82s6`yfiO|@ipL;qoB1~g}($rzdiVZ=q0 z9k2)fGSALLNY4PJ_K+4gM8;AA+nki~fEK?`_=>C#4T~u#PB94OwVACJN90MmoImZ$ zg~Qn5VkuCYf-f+@_*=adiI&2k`YUlU3nzYxhZgYE$l5oMRK6LxPd8PKoq`iItuwkX3@u0*f z6{8ck~@i-1=uCh1G?3(F5o zLuT*n@hID7m|a4tCnj#4OJGM=6;=W@h$sYuL=JnK(lGXj=J;gCE@^6 zw`5vq$U`0X&jHX;XF{G$^ez29F#GiU^z=HZ81F5(pk&eOSp&&2F4&svyO|rEnoY*N zx$7>DKGjcwU@Lf%{^q|j;fIIOotbBhRc4yHcDdQ@fN{;lV~?xt$0efmF{URN>8)Az z69redv_0**T8ef3l`DfgoVUcQ*8`dzO!UFPpn}7@0L|+4I3ptBH9V4+@GFFDo(w(p zgQJ+ph`+TtKcBO5e7Ton7u`lRSKwRzoz?q?&&fJ? z%JmSNBC4Pd?JB>9C+UTao2)di60)+#h{5+g9Qw=OjvoiOOC&wAj(;7q&7vMl-`u2u zt1`vj;pG$N<44|cQ6b$~%_D_+e$Bn%f}-_QE2<6A$*uNqSpzeCMYywTTT4KMNFRm-?I z9W7J&oYgU+^(>)yu7RP{4g;*e_-BkFOr~on8Xegp7ne!NM^Qfn!Xq-`V4;DtwU}M& z3-?IIF-*bAlqkCXi{|dE(Jv>&6s){P{8;QMLAEydnhci?ck@fd&q3t{b2i(qzQcK1 z0pBX!`Z~P%Gj{}3=HBd@O1Jb5e9H6 z3i!q&^;Z~G>%B*+dV3@Tiqh{QKU&LtfoV|Uaj6GY&oW|$X4^UTW1U)ECyJwJTv}#f zls%bC41)d9Z;NSqtdG3E7nzk;-o@nkxbDJoG6~e%SxIaKjG~(s*ti==duM^%a z^_a}A>SuNuHV_P2*VTT@-cCMm`h~PrxMst+%Gb*;a0KI2s={~6wzD6`1()+=DpWX| zl7)^FQDlm7$UF1}cz>Z5Gn;jLbj`=oC&FwdFxV)Hc`Ma!eO{=U#qqsb| z!!2hc7Z!siq(Uyb#v))9}eWGgmGv$X|J-mEO3tfRgP$dEQ1O1?V% z;85UDQ2!AHfuli?p|grWk}DcIqGGTGl2ZIoN+@W6!er^6+q-2q`Yx*Mbce+OO(~XG z_{-Qi2>9PHGMq5jhitRDHj#TQS*Xno(tas)wlaj}>Du?803zWx00Hx~6X(BU5O4xH z-_mT({vzJ7du?Kne8U#9H3Ue2H^5lP+Md~Y0~PV(`x|{Ih|VWG=NQ7BKN}40EVJ9lEm0g;YeGTW|LHKqjwXCxZp<4lu!{UUisf#}&>|`T zgZU8gP9yI}MI%1tzU=jTEB2mm&YYC_8weC02$yqbENHBsbRwqpnXbDy`s&~MCxAe2 zrbzsi#e1621M=hB5amdG`mV#DZ1)Chco zbH2Yjd*#M!=~DSCCi*vMeZDH31G{`TF}hP1@O>T-v7&kCs%rH}5B@kx#T^yR`;zE! z?%}?dfK|VW+NX9FjowG~%7wjB6OO0-V(>|K7K$`6QSD+BkyZcBpnkO|WJHw%4BJ8B zaqJ#^0X6zH0AFcj(^xYHT;a`mbbT(}YQ`o1HlhE3RDrB<^%uakBEaSm7k@bp!f1`$ z;dq2_4YWB?K|l;L$GdBLr`yY7qRuk+>mU_(hCmoT1iQugjz5z~>EV?t{Jq;&$=Nbm z^j`HnHD^VD%ghB9-W)YXK`Anq&qbSm2Zu`O+5ygu!`&kRy^+|eg(!c4Jm~~cvJpQ; zfr#+CvF*tx=+#vf)({={VpPuOw}HcRMcc;9NAhAp!T@(sCB{~??Af)SH8Y_w#=%)V zQf<%s=f1Iv3rS|_{6TK;-h%>o7G?XIu?w~E&9@1HPwJ6}2ssjmLb7V%6X86xI+Yz= zEv-?Mzv9E3{c{~7*$OlvG~q%l-HRr>xr81+ufvwq#JLQ&imY>ecz^DAZ+=HgJP%Lu z=TwJ+zIlNha%C{SnG~M4!}h;nFfNnN`vO=d{d48(z5o>}aK=88LDYh2PNqNy6 zli|%Dhm}e-HQaI=lj*WZRV*vcd?ZT>9#f-OK^fL?eURZ!YNjI5VEMvPLSYtOm3qye zU~1}u!A)1hK>vBjRa#e3x3G*PycFRll%^cc2JMbwl!prX1gIDu-*kL7VGbgd zspbHUh%Idiq*e>d@FQCVX5ycEFG(*}c>0mTLOsx2M4 z@FwpRk2tcPFpVqTZv;AkNrz#KyiIX6N$GM(0ItL6{b-{lHhnBZT*=@w%@ zc4A@NJY1rv`8#iM#BbwphBoq5y7d?l&EFbcY5SFAa&BU5@5O#2+L(5EM1CxRTXgML zC`--75USzN&_Gd}=>PfyBSvrA`zOtQr%EDWnnS$N>O!O2$O2Q7XqQZsu36+48^4Vp{h*-U<@bV|#4OipaHW|wtu{2YJJ zMpd@A#NrK>=$q4J0NZ!8eqFE)5-4Q50&oo`fnJWmoms)tno<(WV(Vd|XAS`uypRvK zQT$P=>w`ScGF7>0z&08$ElA+T1e~Trz^;IWjU4XH5JTe;gs_?+E@H@EP)FTd_PwF(Bz>UY9`ika9?s{{qW6}@DcvEx_Ci}HV z5hdsR=TlQK4x^JuRhjovBR(nem;2XzOQrHvB;H&wndjfzwJ^eSoPP+n-&#e=RAjbA z{4>VDN_xr(eXIy1{J&L3<2c%EkaP?9la@T?ve+E$|Kaqp6c)2-q`PLi9IN=%r5CKw zua#mBII)D%f#J;K19;t5CEm8*#7*up}UQkAx}(ev)>bi?EYqxy}>hiKOUhowdE} z-JR-JPX=p^4^KCc;s+9I}Xfrg_z0x4O@|lkpLTXyUA>UA1a0V;_=%e z`M!T#Adyq~w2LN0GQV$^jfuaZE?s1x^J3gNbcaOziW}9Qtk8ma5Un^V@31mp`*M$W z$bTX%Cuig2$L07(2jt)TjO|^P@sA{<(v2>dP6hERQ@m1EWlC`7&7~j6=^+#wMDZ}H zDtu*cwji6mgd5t@PZTV!Ifx6(ru~z|hU`j4Y!G*eck>9l)QiAYA*t-?)9BJ+r5%wakn-Va~4xnk4f5dPn| z=A@V*PRGAPyV5t!zz$pY|7aQ7OhH76VeVU9{)M@-3pFxj*`=7_m_4yqrr1tF8jJT4 z5T@K^>1xY!F{|7KTh4`VloK@r*wbd6omphNH!DP)tV_Tf=~^yve1g)cx*qa>S;L4>5N6Oop7UC6D z_p7WCH^C%i3l(-TPT@>R^*l6Dj|HcLllOU@>h%{nvuCwj5YS2*=gQ=UtZjnOD3#5~ z(r50)SZrJz!ue(0g}lhAi#gLhPt);dH-pA`I18oeQW$u?0BK^<4*NY*<*YE!i{(xU zuLLzw*5+%SEEbGN!<$WrQ&tDHQUS!&!Wrcqf+6-;>Yc`oOz1a3?{r^)$B74E!_xU^ z4Z&(S1hI?(&E+}Mbh?*HV}sL zmK+vvtA{fI2g&k#PUp5xpN#oG&*7&qf(>WiW&9QAr(^ zfTlN%d}K47%C)}yW#_t$=3uer<)V^^w2ZvXx4z7GcQwVmem-OwfxPC_U{d%7g>!MdN*Vmx{=C8)*dP*Ar zX_@5|!%vg$rt+pY=3T7tD7tj-+^+Vop+{TMS{l|KFEQK+A~`%!21Mu<<`+ORUw-Jy zJPxuyTDhOp%%glKYxJA!8)eRO)RR2r0#aI=zQDq?&xkg zTd}5jC236dF1~i0&9h`N=AR%QqIPA1XhfhK&zX|rAZQFefH%2;eT$e@5@)+=A2aIm zTWG+?OlG+rD9g;0izCJZS(Njy4aOLnDX!EX;w@h|iO&QRyFzbC18mo%5kjhTV>z^e z<0knka^>hCN+&JctGqr^LSxRJ3G&4)7AF5&XWgX??J4#V#Ffv=^NvwCG0G&z#b< zFF*q54Xar&MbrH8)KGRuIR_UszeSumt_}BbgGVT%@sCgQ{XkADqrNTtsZGD%P`a7*8O!Xim=ZWG zBKXmxDlCb6zrbdh#Gc0wD-8%?&f|uEq;P@Y?DA8%TWxa+EJN%^e?JH$G~;_Y)8pGd z!htNDP0EO2k`z%L8N>H>q(mX|NdVoiJO>al>9uvxTM+Fc&#t|)2txP&*|6PI70xyxi`4&dC=`xKKY}mJ` z5gLrL9$k6Zwxcez;}^yQL%&>$mPzcV+y;f)ChVP))?(|hs0g+IhP^9|OWrUZ&-Wv` zJG}xVYCzZpeYz{OP!4q4@h)ZZl)h*A^6%WSg!>;NZH@#$#C~}g+zdi?l_EG)mvCg* zRh~>c+AKGhUqxMpsU1SnE5u}6$*{yDU|b55+raWoGn?EMVnBs8IbgBE!E)vIlF62G zNp!sgqHH;je6MOqqJvN4_if`qm5E;b zZ8P>x=J4Dh%`;nVn!&H(BOxCY5D5uf;4W4r(G{CE3K>M{9jsv+Zgr;Qrm81-|Y5BQu3KkN$yWTdp8HOdC#xtTzde0=vLlDq29bO#C<#5$bSf5=f&L5lx|T z(J7Q0a%5N0VEuHta=VK>11M+J7GrI${)SWE;Qx_q|ogRnaUmv}|v}_LHY z8elj@LB+FrTK_CH^*RuSNH)MMz)Osy{HtMrB2H}K;y}JY^jJRx4*YwNXGQ5;INrXg z6wFeGbE5MvyeQ#oJ{O?31UiE5^}Tl6hTAQzLEXMDVQ}wyLrV$8o}Zy`d)>WzrjLUH zBqNhunA)QWelEQ&A6SxdgW&q}$tpNN1yyv!5w?8Fk0A1}d}Kkh`^c-HhlWV{D6vSo z*Fq^zUrIvIeu!SAup6Mw9T6#KNFHoEZ9;50l*X}e!+S;oi6kIvAH}tEVe$c9&g6J7 z2R{99H%)J1WevFRtC1_U#1n3-CD+i(d$; zoOAm-=HAJByKJUs)xh|o>rk=(1$fqGIXy7>dzO2_v|*k|;4(1Lv=s{CBN3UGw_elw zOH%2obya$MVU?@184mRYuoKN_ofTg26(Y4%K1X9J*4mSGnxcu*Zrp_kvU@^{B0)!* zz*8=jkuMZ#6`837i_%8?^m0@%uPY!Jkf(k)a5>?vqtR?yea#nP>LJVlnVeFw1nwap z(Z)W|pn!B)>#Q5&^Io*F1>@F!VXciMRa0j zE>`(l`FHR!W)!|3cJY?e+Wp1esqcv$xKm=`CpeIo-b-mDjkFhAz;#6A!`}9HRZFoV zkU^f4Tgsh_#0H~;32>upl;a1yMW#5t_oKv8wYkgJ4W1i*{- zePkGm7N-r7FdT0jEDAA;mOjq6!KQXl8)`=(!yehP*7$P;mO*|qIuz}X*nGLo?KWR! z;@l8T|9i8{D^pAsk$xlb01`h}RYY90S!-#*^_?^Lk2F;jo>zMs`^(ja=Jj#ZXd|Y` zsWsT>QHT#_Fs*iH+knQ;TLTg(sZUoMA3+AP`dORt?# zl;GXcrFjp_dSdTd_p&_m=5R+gZJSH&;h}7JBIiX^UMd8d|B0X-ny-Ub)&x-u1(R^q zTN5MaJkodbk$?*kgz~ogo00P0P}b>D<%`fX^Yzuu&AqqIkK=F#Irmwb#lD$(^umY> z&pXhEA{80ga`jrposVa^E=kZouLRhnh=6rv(<3BBM|30GYmYaEEBbTU$s6HYVO^fo1#`&li)u# z8nWPzd;jTaY9%2ecb%cxw$pef{T4W>xB1THn`XSx`>5@Whm5>?P-(*30~3cNGUNyI zFVaqj`cG;7Pn-?@KiRDRlYWZ^AY&Ez7i14iKqW0`_)m2G?)&Zk(PRswegP;F8^g^U zE|?!{le*wds^LyxdXFuqCc?O+yW5^>@1H{XAWc2}98Chtlc1S?$zxKDx7m;U%Wu3w z=3aEh=*IG9bwK6f$b~xY7@;8rQmVmo2Vm@XLo9CQO+r8j800?1p(P3EZj8rMYDv7U zwd8K?bHw)Uhn&X>Yh;u8&(;ny0yebZnK7XEm6$`W|rrt1Jo!hH< z9(|3aE^QV5xd=v0!qystM_jSMY7tDX<+2frm%?l-WGxhHT#g=vu3woBE`Z#3>Fht~ zk!KClZ5ITLI1;_}`!qMW5Oi)o|iu^XBn)6{FDGoVxn3F4Fp3p$Z z_hhA%3?ydGoufmV8V-EFPR^)?bQ z-mGCzDHywxH*dd0zInF30tkkOTr*oGlD81qTz{Q`c*2dd)i1}OAm-fz@&5ezT>02H z?oe#IrXixhap7{!RG=P}#VLw!M!F^FR|Mw&0tJo+^d+G8#nu(&I(9yp5iWY`FAJ0? zsiN?oZN%jzRh2S_V)a*s+TfqnerESd6~yzW*^Ix1;V2w{bI4z$Y=3Z9Tfs`ezst@) zJ4m*oqzq71UuMC02@?5m`Qx z-O)WX7MUPcd;Zf`yILF4)U)O~=#JLleg2Rr{QVLsZi_|3l6bKOr#G5%_3RONlXKK4 zZrA@~5ZOCMC9PmvoyPpvp_*nU0Ys7j@;4+#iRlHHvgVfN9l3jrT2cDIB?0acPF5nf z?)$r&_hHjnkAh?8!e|j~tgEw|lMjPhIpaz${{vLcKcA(u+ZbnM426P&ieq%RI54!T$dL3_=`i@>r@2TbA)tbO5E`ZQ2%oRos zHA*W5>fSQe*?7nSS5GXv2g9>ecdh`Q0~`tE$_658G%4zhT-mb9&rx({ilc+V?=<~5O!pgT89j~FpvQT-jvTBOoSHQl8SuOt- zH&b-J&0FUUsLn^Wk>2x!0MlCvZ4rf74Oq_tl@HBYmfP<>&{E)h4*Cf32P8l6RGi)= zk#9eo0sIk6baUJ4JGoXgl0Vz4b*N#a8T29YsG0XX8}KEQw(Xk1LxT^~%q8v

  • bo z6h?@ta@l37utEmGc|ifd!}Wt@Ww_d+(U}Vt4CwXeNa04ZW`xtqxM}M$;|&MyruzN~ z_$siFg}_&KuY2)^Fg{M=+vfXa_9ZVFFGMx2Q{>e%S`s|U@`xl|3hiE5K2|ocWaR!o zgL_uLyUC=vK0L>^$~gp3;I^fXq%BU}!s?q-zk^%9flSa^@#?H{orCSY`p=%WeeCnH zxoiGc06#0h)SgM$WU~ob1d1l>lR4?DH1lM$k63E0is?3Px;-;WtOOwqLH=?me5A8g zc~-|fjE>_OmO4r}CR1bQB7Ag4I~b!_Itp~)8-pvseCH+x0wylAZFITdXL&>m(TPFJ z%aioDISuDrIq>B;%hiS`i`e8IeljK28G<&KpeHvL)LCf6Anf4#GBH552PN@t`IrrM ztvn!*~d2UKoy?PU}mR=nKf}3esarO!=T@@H*~e z1SOv)@;qdY)s$Q$ZWI_K4K9NAI{3gZ`aEUcn(tpZbWwjntf{@P$36T5qlvRk$A!aI z*m>^oZ>Io};^P&-z)$}GEL$YGedYZNpljvJhz0l?bD)2W20D#gehlyU$IC>~YjGOo zW9E*9TCgQFt>g%`v2r1rkTwH)l=Yiqny6y=5B$Ld7;SrBAioH>*`xV^f|?I$)o~zj zE(l6Gj5#M93%oG^rnul+i^9)&ck93O$5=_q7#2HtFyPaXHQO^z{jvo)uZ*4PcKYb} z#VciIf*?J#ra37{Y$s>P%(!K046&<-xH^M&w4rZ}esCOf$5+43L@ZT@Hzx6T3x_?t zoVYcvF_ll|xEw~gYWu($)>Cyhn5b$RfA=&gE{F9g_WTKu+gFVtlTT@HxjZSMfw`jg zWufi+!)BOPd#A%#V=5?40Q+m}2$=cO1Pfp%-YwXwu_sFo2H%H_wsh+)Q-Z0E`1a>t zqW=H~!8#KTHcPqABu%98rRO=IJl;KKl)1zQ?FSXFj#xTpdjPj!L;7t zlyCFa6jT2I=Qt(?i~KUoN<~7&d>MbjBq!_9424*Ld~5- za<@r%PIr%lx(rkZn7q>`c(X!HT;(l>I^_8`kr#7(ddZiO=Qg32xM*bJh+7msL;!Vy zQY^T(jdq{M7$Gj+k9xdxU?I8Q-ge*@_>&SH7>wZk0vEhqAeTu(rru0=)&Bqk1`kW^ zfNRya1RC{;einUUJfq6+ILDgp$4)Q=IZnH9^-sAb8B^I;rb=G?4niMXFA}T zaS{QGhmi@Ea~r`yq3^~Xzs@r*3!Dz}yvHNfK{`7B0NmY@^M)Jsw*c+OgC{tSn5usV zpPV+Yg_|T#4!YJXT{(ByRg;3|(6gWjO}f|r02!|r#OV)O+5^jsK)WPp`P!5i8mE`2 zN{1Tg_q;q06{u@8&UNz!@1r#_)Nb9#JZl10Qj9jJ_?Y09!u{R;aIKc`zZp(g=z77L z{IH$-OlwctvF_kw*}au#JZ}wvRG@v}`M-^yN#2M$#84wsd8ahu!@ZS#CFB-WD~~aDZ?wyUW&b z2w{y8*&Ii83fHD6m_*v9_2XfzNaf8mhg|;vLlqxBBdoo8xVT&GwO~xWtGC`6@zD+K z-sX>)oK6?$LMJ!*4h-s*01O@whD&eow1Ax*zA#?F*SL=%#xq`_;vTL>;ld-mp#K1P zt@-;?Tk2vO^m0w@k5|S8`yF@q$>?$eOxm!d707ygu4t5;@8{s(c$GDkU{8GO1?_SW zw}s_0f?@_2qijYNu3r^0e9+DGQ~b-p+B z$QX^L>hoF?ESNSxfuZAeV4ju{^4H0;j9w|DbSw6d`^(QYWD3C$0@F?G?d|Uu)IE8^ zv%wB<)g-8e9bDFdIU89ZZ>;r)?}t2V#%)i=TYU4s_nbAR$Ngr^jl9pfh&=_0`vaW6 zl$`0`)>I^}G^Pvqke7y&(~RTBVt|@>YxuVM&$GiaBJA(LM{K%~> zOV2&zU1t?v+!&b1Vd%^K8!WRCWHVp|E(zTnt|4BSY53{@NU-X3VT9^#pmuGp@=kL{ zoi5AaoaftPr&qe3nBCJ`Gm`i&9L^}qdvIW3=3pm(4jksf)oP_eauadZ0txL|M^dmu zl@2gYqw>_!3>&A}b=*&;X+xl33~u)MG2AFLKqL|!2MmXJ zyXq7Yka41NVc(6C6d>=P4Q0Xk#qc1X&~u2){oz$)ddEFTr67b>&bj*%Xl8UbHQo#N zeQ}OLm?hSnHwkrGf`yM$94V)Pc~Q1Vn8l^}s42DSW*~?%H4}BrM{M&Q!`Vfv)Bi>Tr6&3HEc9;f1-K zt`OIZPtYBkT>a>H!GF`PI18pon6ruuj)NvOKNJOEBK36p$_eLVsHd3t=Oeb;M6Fy0 z{KlgcnYsXb*Bng0b_K*hL-ALe(R9oW&!mwq2vsv3V|tkkDsWQ{(tXtZQ40P_ig6F`W(0Xuk@#B+2kzj{#n$Z?J6 zI*anl>sVOcJ6ewbq+BnHFCo#ttPtHgXAtA89X2@X;=pVxmU-=QT&Jf3hO4uy>kWJl zQy2}iM~s$|y7wm!ExEO&Pxb9_mNFD-x&J+P~a>v$7!Y} zp_b0rt=<{VYU<~+a%B%2DyPxSe)4`ZTf@-6Fhu7OCfdX_*0g%a$O`e#-YgjW;w!GT zn|PyWE|NGkz)^G8kwgr3vg``S{mH~`3CzEK&uE~Le&JR{_ zFdnu!qe0y1a%20)A0FGACLMGz9gA)T+BTKN4NysTYn#PZu5xB%TKEbef@(!;{jRa`nAab!^)w_99w zpE$H;6KkstcJl8I)+$=i9-y_tn_WPz#-7|BKNucKI?fqRxBO&!*wKRiJITqUm)=GJ zPBg#HMFe%;RB-F)=K&;*6wm&aHzKD`S0V%r`xu*MD9vw!HZCIQQvJKZ*6*{s^@%S* zyPI2sUNBuYwSbCmt>*y5p>{3CK-pMphN61K8iSSXvpwSA+D@p7Vg`t({C^lG`oq8G zc@j?g$~E|BgR$27$xtRA?AvEpLE_^inYt8L<>w=W^_>ZPl;Ru(CT@#C)K36QTCZLw~|<=>Gsr z=%hSBvAJukg?yUIwO{S>gWw07Fjw1E0L; z_mAsVYRevP2UkZ2=nnn-XB!HxzHpeCl)pC=PqDe{D<>%b02mi%Q`SogmY@%Ly1vuG zJihSL*TFDs{{TOo*^(I}2HI%XoK*^|#I5<6B9J17yPwXpO(Rc)DKUvVlsrG6|>GKdZ1tcJ9R=}qM#_S z-j~3~3Fg}x0RVe>Tve|i@6Fa<2;!4~a(D5X2Vo9y&azctYkau@eWQosnL4Al#mYH( z;JWjIgEbJ|6FMf{??KKBWOTsi&IQIT?0zx1W3{bx2G!`w2>G-KKJOi3%S2yp-TKZH zr*DJ2eQqFkv?ta%kS6O;6FA1u_onZ_rZldU4kt&Q;ESvaQJno{{{R{Y@Fol|E;gwb z%V#-vy{H;A*>uQ{dNi@waz1gG)4~90;uZPIZVHW{s=F?{XCT@~VHnfJn9jJ1Y5q{S ztU8Yd6xW;)ZM*f9p03PM3_t$Xc1%yX)0+4);(1;kFzkm72mGc+b|XFk@~HAa0Ii)+^RqT8g}= zS>q?z+|8n2X*}X-eFO;u-27w+=*G$cz4e~YTG3wnUJPDtyLAS6$@+=DlgZW?C!ZZ$ zaoP$$rzCn+F*z2hk7(}ZlzVj?5IEj9c7k}0&VKWRivZr%$J9z4L=;~R@vl~u-u8Yj=4;5Z zb|$HP;;_;tx!Js$oPU#P(IcTKCb3VzY`X*BNxrbh>Hh$!h4OJL?|23c*Bs@y=6Z_! zg!;`ypGZX-4H;I^SxZ743#<#G2&m18^h`9Rmg8EffR*Y`UT}9%)7$42Kcm(Y#Y)cg zyA&L<#-xaN)ndCj&ZW+ow(a{K99G)Z<-jQo9-_FJGcAoUd~zTt*(fFpO-Yv>70{Wul|?zig<8KCE_dCKAp zQooF>R{MJMtli~3eI#`bkn1=BSS;!B zj}f&`niKlMCm`|%X^7*yU_?uLM_AISZHHbA+B-9opYbH$;{6v=gKMuR_{fe%wt)cf zrz{IyfPrYc`hIhT-h;M2MK6puGT(~;zDhm`l+u4#a6a8@DWuJ!09n;}F$yRKyaFEe zsm^QoXC3N%0x!I6fA=^JUQEhM$obz`=Rq5(mUUuu-#J8{mk`C$O1RL3BCowT>>ZH5H~^vwo)cf= zx>A&6!)1 zn#aHmo-oJ^<~tMTa}~}@D@-@s55_0=4_)Yx?Q80oIH9f(>6MsCguD-Q&i&)?p=TXj zy1$Lhx*R;-F%b!C?Rs)W>qA@|Z(QVL(XqF9CD^laIpBdEoZ-@a1on(oE2-Q1I{G1q znl#wwpp%@MDvcEatOoE4dZq8Z;vzMokypHu=qM> zTWd_WT5wi_X>R;O3G_b(jQZ2HT2W zn=ii9?;9f-+WC3Q+m&;iM+)Iw;8qIwF&`L6lN?GDNs_x^+ky;i{<21kMB->La_mt& z4j`FP3+DyXtv{*yIG?Pd=qTKr+<$)4m!!#`(!0u8P5vU za|G7WtnX8GUd?&_Ih@+{SULV4KtmyAh=PUt0*T*=7wJ<{2zj(|P^Ts*e zn-h(7i(+ryd&Bf`{RTJ={Y*En3>0IQN$7BVZv$Ma4zscIHGx1;a0jY^-nkAr=R-@k zsyH!4Jis(JQmygMD}_PYby4SO4e^YaYjibAIig{JVQL^E`QsIGShyk8VM4Gb5$4gnf;0SOzrOIH89=MgxpOoaST)52lS%p9dUiJEASMy?1Rzc@xj*nwgfL^+wEO>vr)m!yZRt#Bx&wVvT?*yKdj zebR*ISNp`an4(c_Jg>tTuV$qP0qpCn(gLBRidQxOlWz`=`t6E=(wufpSBIQV-JCD` zV&U+*H<4!VM>8GbX@`fCI!^;RZ+zpr&WGn0vgfkUk--Og#LP`rq~Qg(C8+s1xp}y4$4Lgt==sX08js760S>;g z)Y|vw8<+N^c=d{>2~p#IGAafeDk5y`!27*@Fz5hALNuF6aa^OlH1(~2cu_r(X!Aps z(*uq4W`l)1>s!TXOckw_-}3Q{?%^`F4k)3<0`URdh=dJt`o}SOfEE=FjxZbHz>Q*l z9btQi!}fQhhjbTol#$f2aCf0%C$UGd;X1^f5U-Nz{EA?$5)YfYWJqY_X97p{ga*M9 zV4;9w?)l4g4xA0}$*vr;{xZA}9H6`e#O)khk&5Xs3QPyqow#_)mwOYjG?;Q9iLKpv zZk{;8P2GQ>ckOO1Ssy}i2nsWC(oS#tc;PbudPa$iZC#b?uf z`|PorCcAwsFvS?$QEiTb_MS2#njHx+#hv_~PW|`17*5psjIXAwF3Joe8mz60Vv!7J zXY-Y>wI%&ZXV!FDX&`w?P7xuqX^S?)yc$R}?QKuPr zqX*L4EOZGJYpH}BDMFB9_U0NTGyXyNzmxcl!cW5j*ka30U)ONIBG{!O;D-hrq2ogg zDQ^;&qQ!0I=whdj)qOY0vkDR{QP#w8a+8cqP^XlYp+v8+Y+4j3>kbn-!yZehT2_Rm z1_^le_*R2WG2)drL=Jq5TBfnjs>w)(7?EMXhFxl@x|g0szs$B+{%=2;$5*VFBFpQ( zp>;HR*<0{x{M;CDOx=&lw4Ss@g&he)VJ5o#b^<7d3AtGV+>995>TpZ7jTI@Pn#vPo zu3r=)8Vz=FQyMX16(sk5U-k}4S?DiA6dW<+LxgnX@wpT|iDW~0LXWe0-MU^JEK_MV zUg-6jO5o4apQ+B{1%h^Em$|~l3U+LAu;RBcf_KAsW{IrY4CF*L9z0we6iJg-KShI5Q*hx_~DX=Jc zP4|@>B$GG+vta9L9xw6^B5_uQ9eVX)hQdXhi07(AF$22Tso?46?Fub?L6ef?AmN3pOXA1#8ZN>EDQbLu2Df1=j`!8+kCaA9R)_nj-zdE@I z%BkUtJ9=J~U2Pqa(ieB@)`|ehosiA%bAF+?)YnP%I5EMMa=cbi*qy1Wlut(pwU;3L zYv7|S7+jo_XfYm+Rh)17_2AdV80+XF%VHlv>uf+~;=_}R1&f;bsm>R9&D7sIWpT)r zxH%rLVuKx4DbUWWisvUOH%-;NGjU_iU`I*6e!WsPYlAn|8EPT2c7*IhHm^@3jXJt$ z0wrl&QR+!@V=1%*@$4@jx z{^f~up;5UEg*iD&EAsoIE?+P2l4>N~hKrP3Dc0d1Pq^(qHzI|Xa%_IvkHU869T1H| z6zm=LqSm3=5kiGI5Y%wvCvLl3-7AQTvS_%G)QvAM)f1*)`lwSWd8&OQsxq^rqjt$S zc8ILpqS3*{i$+L}Rln&ED3AD63!C^&!O=}9;L_Ue*l}z`h`kVxk5|vp9ZqjA)BN19 zZMuCg_@Wd@dW%({q3S{TDs;u%RXXO*94#G)yEh@CKi>3J3B~8PAA64(Dj~te-)?1I zm43p9a;s2^onJLuv-U$qxVP1DQM*eB;Ve5bMNphuZL@6@RVz(+WAM4YQGHMPnjG3H z^(*$ZMY?FIl`hXz()L^wRIDx>HO->s+H38m>~7&9LM|jdn4#iAgt3a1hY4CX8$~vs zz7pD&6#9h=c~y6KTorDj{t*rlxVH%9BOMg~Bonk4n6+>!unc)On!%f8?@lxO+9%2o!*OuA@#wj#GFK>fo9mE z#x`}FOS;n75#_v7iE|Q;+-^dK(oRuPv$S3I6w;1*rjA6VgR*evBq}zN?@b#z?Z!89 zm7=AP;S|!eaFUhVXk{E6q`xb5Cl#7mIY|m0l~1!yBw0yI2MbXmMYkKvAsJk13ThFN z_v)%AO3t@6oo1HgZPJ`utx{3RqVKSZa+WR^WJr-B71W&+NSuUuF3=H9`xc6J4jie} z=JQnt-BeY%Rc=bnFR0kqKleQcw-W62cuB=7T-Pf5p5fN<3$Xd#)v-q&}SVU8= z+845JJjuhk!YZNbA0z>?^lmS{naw&?o`XE2Nt;q^&zPtvvHDw9ipf$ zY9%J^ICG7=GWJdvV;gI&4RsVmTb>}U#;aY&ijK^3jikH*Z<_GN#5|~*cT*ul%$z$K zDOs#9UJJ(Ue^kL@XIIY#3Q*PVHRIe9%qN&T|of1w> z90!7LFL(FLRmVLS|HJ?x5dZ=L0s{a80RaF200000009C30}v4e5)%|4Fa<#W+5iXv z0|5a70R2@yIiQ3DANQzAX^x155UjWLek)h|>stJ3pje}1`g(U#V$nO>$NaU|t~bBI zDlfgD>O}3>zZlvT5$&XpvVc2w-{y);QKba^XcnOnEaVTi{{U*!8(FhEc7#F@;LX-K z75<4;bp2^U3Xr(g&HxmiC&N@iNhBc<{!3CwkC{7eks|=!~i^TQrrWtxzm4XDdJl3Bzucu9EVxO*}tTRs&HOgx?6BMb# z$F0!KDTXXc{CpRP+#*!rQ&VdW7IlOyFT|mVCHw7NUKm4zI!8s~5k#as+sBE8igyI6 z6OE{vqNwu7Uwzb}Qw(ur)Z>)0!&&0;?By>o^T}ASl{o883|L~r61wYzJae^lDHjGA zcu^4dFcPepToc<}bfIHmL{Y^|7qF5{rpk<58OR7>1nw)7m1tt3#GP1}lY3KIQ45nvl1MTuv>y4h?@xG@CoTYZ*rBJsW|#EBG>r3(sHn$?)`Jaa*=r19`H)`VJ%HeyOD zuseMgC6x2u`Y}q%#On}NC`FpK7_P6;(zMl&?2e}Rx$R`%)B-11g$AB1+C&N?*=3~+ zK!VxFhCD?soQH=l3N>}sykCd#j}p!{x+NfDt|1Yl*lg~FCXP-Q9LTWE zRSe>yfeA2+0-#SdQyiFANcF^mG`1{Mrkq=1m}ZntU|85z-xhY?Y+DyBgd7-GRK$Wf zuq}G1xk+yxfJ_ljZedIihti8Zy}h3Admr21*=$&16Exz8gJiNoQ8f2@MW-7t$T;l_ zcJ8m;-A?;<-9|juLk!}?`gKtWwsfZt3|MC&Q}m+km}1IZVXw_a_LwIs#R@mEgZd;)E*B`2U)ivy(RqI< z?doHjT?Fi=KjwJ99Z^cJ0BwQS*>X5n$Sc3?MH4S|hbg*>3w@4<0GNZ+WI`k>@6J7h5mJ(qr4rzY6 zBTJPUv@aKziryJhLTn`VI5<>FN-RfSdP59Yb5T;IMzCUF#GHc>CgFcgoNo&%vZ9Z) zgw{{msF*{Bdg3lOT^znH+`#*1NaGqaaZmrm05}l<0s#X80|f;K0tWy9000000ulod zA|NsoFhT(#BQa705kXO5GeBZ-fsvuX;S?lLQ?b$U|Jncu0RsU8KLF;lZ7TXoo{^5; zuks?zvKIv3}i78x<=EG(mf_Kq8ZsNyMTLm6#Fg=7<+BmwN5Ug% zG;JbQk;KJHRC24{-^ceGw`!G}b9;}(D7_?GA!QvAZ_iV5@iLdn@?t-ck?A^ZEuXJx zCE?Kh{$j3G7*+cQh>)-DxS2mb|Tr6A~+t)M9ZMRame7^5lAnTOu^i5txdL_EkzWX~jm7i7rCMM6^cI zXp2pG*xRU9Q!6%{wZ@s$Wuqf0Hia_e;wr82va*=RRE8Twn)yb`t0Tja8Y-f$hP;T9 zQB;vF#v^E9n8=ousfj_lN-NWOF(zU}RO%$h6B1b3BSeuQZ5=;^mMleCbo)I@e^FJ1 z)YSHk5~oe^BCOVE+bJ0lSzi+&i3%;%Pe|62OH67E+`NwtR>)}TRXCeU&9z2W$dQhQ zY>Kko*i>4P{sxp<*QXjh0)Yye4=BoJF!xx;zUfP@){!Ize6bMTT1OM4SYXDGTVz%k|k}}l8Hi)q$)sBt4j}t~LWoD4s4NX_-Wvx_J z7;d7YRFKP0A$0VZ=va|qMHE#OlU;^lR_gc?#Aw<}s!b9$Wo(A@jNd_dIO>+Hti~}q zuZfQ$L|0#lAGl>{MC!ga$3$$R*>iZsujH5+*uou2j3l>R$4MPi9MIo&?7Pa8_bE}9N~LO@BGKLEO8iH(n#&$` zCs#5gl&IZ3gjy=S){ZN9tbTUL$dZy?cZbMr>Pd7&i4sFK;&B>R$Y_ybJhod!OpJ9! zh_uqua@9$vqzDMl&N`=-YX4eW+Q$Y$Kp#%b(t9R6;~omHe98tm&B=0a*D#R zw_0O1>*Hv#wA#d#xpk^_4N;@;>>|IGjCEsaEL>&V-c(c6qWVgW$l79VG3752OooWj zS$UhSk1Cx`pLc0fr@XD{Dtbm?soRLFFNT>@iIFZ+(Pd^S{)dMp^Xw$1?nYYjjg^#( zM?_f2ks8tE#lC{G9vd-F+P#a582h_PbL7_W$yUqA@mk8u8ltNmL|BbQZc8xH7BV7q z{5|LSPw?L#=u2K#mb`7^HKll+Mm&g;8B|syb(XBTZN`jBYmc949H|l7ODoAnkkT8g HTOa?~NR~Zl literal 0 HcmV?d00001 diff --git a/website/templates/company/team.html b/website/templates/company/team.html index 8b4c4e26774..fb68be6af08 100644 --- a/website/templates/company/team.html +++ b/website/templates/company/team.html @@ -71,6 +71,19 @@ {{ _('Senior Director, Business Technology') }}

    +
  • +
    + + + + +

    + {{ _('Mihir Gokhale') }} +

    +

    + {{ _('Associate, Business Strategy & Ops') }} +

    +
    @@ -85,21 +98,19 @@

    - {% if false %}
    -
    + -
    +

    {{ _('Brian Hunter') }}

    - {{ _('Account Executive') }} + {{ _('Account Executive, AMER') }}

    - {% endif %} - {% if false %}
    -
    - -
    +
    + +

    {{ _('Anne Krechmer') }}

    @@ -140,7 +150,6 @@

    - {% endif %} - {% if false %}
    -
    - -
    +
    + +

    {{ _('Claire Lucas') }}

    - {{ _('Director, Global Business Strategy & Operations') }} + {{ _('Director, Business Strategy & Ops') }} +

    + +
    +
    + + + + +

    + {{ _('Shavoyne McCowan') }} +

    +

    + {{ _('Executive Assistant') }}

    - {% endif %}
    @@ -224,10 +244,10 @@
    - +

    - {{ _('Richard Raposa') }} + {{ _('Rich Raposa') }}

    {{ _('Director, Global Learning') }} @@ -260,7 +280,7 @@

    -
    + -
    + -
    + -
    +
    @@ -322,7 +342,7 @@ From 279936632aae8e2ecc790e3c5c1b87028e02def1 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Tue, 12 Oct 2021 16:15:24 -0600 Subject: [PATCH 551/950] Add webinar signup promo to homepage --- website/templates/index/hero.html | 23 ++++++++++++----------- website/templates/index/why.html | 4 +--- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/website/templates/index/hero.html b/website/templates/index/hero.html index 873bcf9487a..79ef8974ca1 100644 --- a/website/templates/index/hero.html +++ b/website/templates/index/hero.html @@ -1,41 +1,43 @@
    -
    +
    -

    +

    ClickHouse, Inc Has Arrived

    -

    +

    {{ _('ClickHouse® is an open-source, high performance columnar OLAP database management system for real-time analytics using SQL.') }}

    -
    -{% if false %}
    -
    Introducing ClickHouse inc.!
    +

    ClickHouse v21.10 Release Webinar

    -

    ClickHouse, Inc. Announces Incorporation, Along With $50M In Series A Funding. New financing will allow the open source success to build a world-class, commercial-grade cloud solution that’s secure, compliant, and convenient for any customer to use.

    +

    + 21 October 2021
    + EMEA, Americas
    + 4:00pm UTC / 9am PST +

    @@ -43,4 +45,3 @@
    -{% endif %} diff --git a/website/templates/index/why.html b/website/templates/index/why.html index a4a3c603168..58bb42ffd31 100644 --- a/website/templates/index/why.html +++ b/website/templates/index/why.html @@ -1,8 +1,6 @@ -
    +
    -
    -

    Why ClickHouse

    From 4be8cafc5c80b5340d7ac4faeed42dfe87138e5a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Oct 2021 01:59:12 +0300 Subject: [PATCH 552/950] Amend --- docs/_includes/install/universal.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index db1072f149f..7cba682e772 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -24,10 +24,10 @@ then DIR="freebsd" elif [ "${ARCH}" = "aarch64" ] then - #DIR="freebsd-aarch64" + DIR="freebsd-aarch64" elif [ "${ARCH}" = "powerpc64le" ] then - #DIR="freebsd-powerpc64le" + DIR="freebsd-powerpc64le" fi elif [ "${OS}" = "Darwin" ] then From f08dd0e8d8c475c7949a23d8b20499509e0c298c Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 05:13:34 +0530 Subject: [PATCH 553/950] modify flags definitions --- cmake/cpu_features.cmake | 37 +++++++++++++++++++++++++------ src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnsCommon.cpp | 2 +- 4 files changed, 33 insertions(+), 10 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index f0750c69294..44a804e64af 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,8 +18,10 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) -option (ENABLE_BMI "Use BMI instructions on x86_64" 1) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) +option (ENABLE_BMI "Use BMI instructions on x86_64" 0) +option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 1) +option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -129,8 +131,8 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () -#Disable vectorize due to llvm autovectorization bug with avx512 - set (TEST_FLAG "-mavx512f -mavx512bw -fno-slp-vectorize -fno-vectorize") + + set (TEST_FLAG "-mavx512f -mavx512bw") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include @@ -143,7 +145,7 @@ else () } " HAVE_AVX512) if (HAVE_AVX512 AND ENABLE_AVX512) - set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () set (TEST_FLAG "-mbmi") @@ -157,8 +159,29 @@ else () } " HAVE_BMI) if (HAVE_BMI AND ENABLE_BMI) - set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") - endif () + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + + if (ENABLE_AVX2_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX AND HAVE_AVX2) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") + endif () + endif () + + if (ENABLE_AVX512_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX512) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw") + endif () + endif () + endif () cmake_pop_check_state () diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index a84db5701a7..729d82da41d 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -269,7 +269,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result data_pos += chars_per_simd_elements; filt_pos += SIMD_BYTES; } -#elif defined(__AVX2__) +#elif defined(__AVX__) && defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 6bc596008d8..4793b6bd9d5 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -346,7 +346,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s data_pos += SIMD_BYTES; } -#elif defined(__AVX2__) +#elif defined(__AVX__) && defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 804d00d4d7c..ea5ca4e45b4 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -270,7 +270,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; } - #elif defined(__AVX2__) + #elif defined(__AVX__) && defined(__AVX2__) const __m256i zero_vec = _mm256_setzero_si256(); static constexpr size_t SIMD_BYTES = 32; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From 0ad20e661329525c5385de4a43d0a0af94544a0e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Oct 2021 03:14:44 +0300 Subject: [PATCH 554/950] Fix test --- src/IO/ReadHelpers.h | 14 ++++++++------ tests/queries/0_stateless/01888_read_int_safe.sql | 4 ++-- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index bfb30e8b95c..5077ee2df21 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -276,11 +276,13 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) { case '+': { + /// 123+ or +123+, just stop after 123 or +123. + if (has_number) + return ReturnType(true); + + /// No digits read yet, but we already read sign, like ++, -+. if (has_sign) { - if (has_number) - return ReturnType(true); - if constexpr (throw_exception) throw ParsingException( "Cannot parse number with multiple sign (+/-) characters", @@ -294,11 +296,11 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) } case '-': { + if (has_number) + return ReturnType(true); + if (has_sign) { - if (has_number) - return ReturnType(true); - if constexpr (throw_exception) throw ParsingException( "Cannot parse number with multiple sign (+/-) characters", diff --git a/tests/queries/0_stateless/01888_read_int_safe.sql b/tests/queries/0_stateless/01888_read_int_safe.sql index 3caa4878aba..3aea8e38ab0 100644 --- a/tests/queries/0_stateless/01888_read_int_safe.sql +++ b/tests/queries/0_stateless/01888_read_int_safe.sql @@ -3,8 +3,8 @@ select toInt64('+-1'); -- { serverError 72; } select toInt64('++1'); -- { serverError 72; } select toInt64('++'); -- { serverError 72; } select toInt64('+'); -- { serverError 72; } -select toInt64('1+1'); -- { serverError 72; } -select toInt64('1-1'); -- { serverError 72; } +select toInt64('1+1'); -- { serverError 6; } +select toInt64('1-1'); -- { serverError 6; } select toInt64(''); -- { serverError 32; } select toInt64('1'); select toInt64('-1'); From 798d8c7c1772eec4ca679c0f7e42eb5fc3d2b6e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Oct 2021 03:16:09 +0300 Subject: [PATCH 555/950] Fix test --- .../0_stateless/01425_decimal_parse_big_negative_exponent.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql b/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql index 7d0993c1bfc..7f276d1f8d4 100644 --- a/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql +++ b/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql @@ -1,4 +1,4 @@ -SELECT '-1E9-1E9-1E9-1E9' AS x, toDecimal32(x, 0); -- { serverError 72 } +SELECT '-1E9-1E9-1E9-1E9' AS x, toDecimal32(x, 0); -- { serverError 6 } SELECT '-1E9' AS x, toDecimal32(x, 0); -- { serverError 69 } SELECT '1E-9' AS x, toDecimal32(x, 0); SELECT '1E-8' AS x, toDecimal32(x, 0); From 80760c0a0c66711757fc087679112df27a57a9c3 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 05:59:20 +0530 Subject: [PATCH 556/950] revise compile definitions --- cmake/cpu_features.cmake | 28 +++------------------------- src/CMakeLists.txt | 7 ------- 2 files changed, 3 insertions(+), 32 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 44a804e64af..c5a6eaf9c8f 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -16,12 +16,10 @@ option (ENABLE_SSE41 "Use SSE4.1 instructions on x86_64" 1) option (ENABLE_SSE42 "Use SSE4.2 instructions on x86_64" 1) option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) -option (ENABLE_AVX "Use AVX instructions on x86_64" 0) -option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) +option (ENABLE_AVX "Use AVX instructions on x86_64" 1) +option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 1) option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) -option (ENABLE_BMI "Use BMI instructions on x86_64" 0) -option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 1) -option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) +option (ENABLE_BMI "Use BMI instructions on x86_64" 1) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -162,26 +160,6 @@ else () set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () - if (ENABLE_AVX2_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") - if (HAVE_BMI) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") - endif () - if (HAVE_AVX AND HAVE_AVX2) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") - endif () - endif () - - if (ENABLE_AVX512_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") - if (HAVE_BMI) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") - endif () - if (HAVE_AVX512) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw") - endif () - endif () - endif () cmake_pop_check_state () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d5977b31159..cac5b70f489 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,13 +284,6 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) -# Use X86 AVX2/AVX512 instructions to accelerate filter operations -set_source_files_properties( - Columns/ColumnFixedString.cpp - Columns/ColumnsCommon.cpp - Columns/ColumnVector.cpp - PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") - if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) endif() From b94aa5bc502580b903b12d155cbad907a9a54106 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 13 Oct 2021 04:59:18 +0300 Subject: [PATCH 557/950] Change the link to GitHub --- website/templates/index/community.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/index/community.html b/website/templates/index/community.html index 5b2f8b8f769..08ccf0c7580 100644 --- a/website/templates/index/community.html +++ b/website/templates/index/community.html @@ -25,7 +25,7 @@
    - +
    ClickHouse GitHub From 4b080489b5a15d89bd6b8b84aeae0f23acc5ed06 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 07:34:21 +0530 Subject: [PATCH 558/950] revise flags definitions --- cmake/cpu_features.cmake | 28 +++++++++++++++++++++++++--- src/CMakeLists.txt | 7 +++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index c5a6eaf9c8f..41a3dc26d5a 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -16,10 +16,12 @@ option (ENABLE_SSE41 "Use SSE4.1 instructions on x86_64" 1) option (ENABLE_SSE42 "Use SSE4.2 instructions on x86_64" 1) option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) -option (ENABLE_AVX "Use AVX instructions on x86_64" 1) -option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 1) +option (ENABLE_AVX "Use AVX instructions on x86_64" 0) +option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) -option (ENABLE_BMI "Use BMI instructions on x86_64" 1) +option (ENABLE_BMI "Use BMI instructions on x86_64" 0) +option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) +option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -160,6 +162,26 @@ else () set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () +#Limit avx2/avx512 flag for specific source build + if (ENABLE_AVX2_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX AND HAVE_AVX2) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") + endif () + endif () + + if (ENABLE_AVX512_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX512) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw") + endif () + endif () endif () cmake_pop_check_state () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..d5977b31159 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,6 +284,13 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) +# Use X86 AVX2/AVX512 instructions to accelerate filter operations +set_source_files_properties( + Columns/ColumnFixedString.cpp + Columns/ColumnsCommon.cpp + Columns/ColumnVector.cpp + PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") + if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) endif() From 0409b775bf2943295ed31c5ad59dc6bd36fbb435 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 02:11:15 +0000 Subject: [PATCH 559/950] fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 00c37fe9a75..4f2a775c265 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -65,9 +65,10 @@ void DirectoryWatcherBase::watchFunc() #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wreserved-identifier" FD_ZERO(&fds); - FD_SET(fd, &fds); #pragma clang diagnostic pop + FD_SET(fd, &fds); + struct timeval tv; tv.tv_sec = 0; tv.tv_usec = 200000; From 23602f46075821871fda78eaa238dd4a8eac9864 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 30 Aug 2021 23:29:09 +0800 Subject: [PATCH 560/950] Speed up part loading for JBOD --- src/Storages/MergeTree/MergeTreeData.cpp | 485 +++++++++++++++-------- src/Storages/MergeTree/MergeTreeData.h | 15 + 2 files changed, 333 insertions(+), 167 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 8821991bae3..b88a152e1cb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -897,6 +897,261 @@ Int64 MergeTreeData::getMaxBlockNumber() const return max_block_num; } +void MergeTreeData::loadDataPartsFromDisk( + DataPartsVector & broken_parts_to_detach, + DataPartsVector & duplicate_parts_to_remove, + ThreadPool & pool, + size_t num_parts, + std::queue>> & parts_queue, + bool skip_sanity_checks, + const MergeTreeSettingsPtr & settings) +{ + /// Parallel loading of data parts. + pool.setMaxThreads(std::min(size_t(settings->max_part_loading_threads), num_parts)); + size_t num_threads = pool.getMaxThreads(); + std::vector parts_per_thread(num_threads, num_parts / num_threads); + for (size_t i = 0ul; i < num_parts % num_threads; ++i) + ++parts_per_thread[i]; + + /// Prepare data parts for parallel loading. Threads will focus on given disk first, then steal + /// others' tasks when finish current disk part loading process. + std::vector>> threads_parts(num_threads); + std::set remaining_thread_parts; + std::queue threads_queue; + for (size_t i = 0; i < num_threads; ++i) + { + remaining_thread_parts.insert(i); + threads_queue.push(i); + } + + while (!parts_queue.empty()) + { + assert(!threads_queue.empty()); + size_t i = threads_queue.front(); + auto & need_parts = parts_per_thread[i]; + assert(need_parts > 0); + auto & thread_parts = threads_parts[i]; + auto & current_parts = parts_queue.front(); + assert(!current_parts.empty()); + auto parts_to_grab = std::min(need_parts, current_parts.size()); + + thread_parts.insert(thread_parts.end(), current_parts.end() - parts_to_grab, current_parts.end()); + current_parts.resize(current_parts.size() - parts_to_grab); + need_parts -= parts_to_grab; + + /// Before processing next thread, change disk if possible. + /// Different threads will likely start loading parts from different disk, + /// which may improve read parallelism for JBOD. + + /// If current disk still has some parts, push it to the tail. + if (!current_parts.empty()) + parts_queue.push(std::move(current_parts)); + parts_queue.pop(); + + /// If current thread still want some parts, push it to the tail. + if (need_parts > 0) + threads_queue.push(i); + threads_queue.pop(); + } + assert(threads_queue.empty()); + assert(std::all_of(threads_parts.begin(), threads_parts.end(), [](const std::vector> & parts) + { + return !parts.empty(); + })); + + size_t suspicious_broken_parts = 0; + size_t suspicious_broken_parts_bytes = 0; + std::atomic has_adaptive_parts = false; + std::atomic has_non_adaptive_parts = false; + + std::mutex mutex; + auto load_part = [&](const String & part_name, const DiskPtr & part_disk_ptr) + { + auto part_opt = MergeTreePartInfo::tryParsePartName(part_name, format_version); + if (!part_opt) + return; + const auto & part_info = *part_opt; + auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); + auto part = createPart(part_name, part_info, single_disk_volume, part_name); + bool broken = false; + + String part_path = fs::path(relative_data_path) / part_name; + String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; + if (part_disk_ptr->exists(marker_path)) + { + /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist + size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); + LOG_WARNING(log, + "Detaching stale part {}{} (size: {}), which should have been deleted after a move. " + "That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", + getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); + std::lock_guard loading_lock(mutex); + broken_parts_to_detach.push_back(part); + ++suspicious_broken_parts; + suspicious_broken_parts_bytes += size_of_part; + return; + } + + try + { + part->loadColumnsChecksumsIndexes(require_part_metadata, true); + } + catch (const Exception & e) + { + /// Don't count the part as broken if there is not enough memory to load it. + /// In fact, there can be many similar situations. + /// But it is OK, because there is a safety guard against deleting too many parts. + if (isNotEnoughMemoryErrorCode(e.code())) + throw; + + broken = true; + tryLogCurrentException(__PRETTY_FUNCTION__); + } + catch (...) + { + broken = true; + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + /// Ignore broken parts that can appear as a result of hard server restart. + if (broken) + { + /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist + size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); + + LOG_ERROR(log, + "Detaching broken part {}{} (size: {}). " + "If it happened after update, it is likely because of backward incompability. " + "You need to resolve this manually", + getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); + std::lock_guard loading_lock(mutex); + broken_parts_to_detach.push_back(part); + ++suspicious_broken_parts; + suspicious_broken_parts_bytes += size_of_part; + return; + } + if (!part->index_granularity_info.is_adaptive) + has_non_adaptive_parts.store(true, std::memory_order_relaxed); + else + has_adaptive_parts.store(true, std::memory_order_relaxed); + + part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); + /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later + part->setState(DataPartState::Committed); + + std::lock_guard loading_lock(mutex); + auto [it, inserted] = data_parts_indexes.insert(part); + /// Remove duplicate parts with the same checksum. + if (!inserted) + { + if ((*it)->checksums.getTotalChecksumHex() == part->checksums.getTotalChecksumHex()) + { + LOG_ERROR(log, "Remove duplicate part {}", part->getFullPath()); + duplicate_parts_to_remove.push_back(part); + } + else + throw Exception("Part " + part->name + " already exists but with different checksums", ErrorCodes::DUPLICATE_DATA_PART); + } + + addPartContributionToDataVolume(part); + }; + + std::mutex part_select_mutex; + try + { + for (size_t thread = 0; thread < num_threads; ++thread) + { + pool.scheduleOrThrowOnError([&, thread] + { + while (true) + { + std::pair thread_part; + { + const std::lock_guard lock{part_select_mutex}; + + if (remaining_thread_parts.empty()) + return; + + /// Steal task if nothing to do + auto thread_idx = thread; + if (threads_parts[thread].empty()) + { + // Try random steal tasks from the next thread + std::uniform_int_distribution distribution(0, remaining_thread_parts.size() - 1); + auto it = remaining_thread_parts.begin(); + std::advance(it, distribution(thread_local_rng)); + thread_idx = *it; + } + auto & thread_parts = threads_parts[thread_idx]; + thread_part = thread_parts.back(); + thread_parts.pop_back(); + if (thread_parts.empty()) + remaining_thread_parts.erase(thread_idx); + } + load_part(thread_part.first, thread_part.second); + } + }); + } + } + catch (...) + { + /// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad. + pool.wait(); + throw; + } + + pool.wait(); + + if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) + throw Exception( + "Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", + ErrorCodes::LOGICAL_ERROR); + + has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; + + if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks) + throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, + "Suspiciously many ({}) broken parts to remove.", + suspicious_broken_parts); + + if (suspicious_broken_parts_bytes > settings->max_suspicious_broken_parts_bytes && !skip_sanity_checks) + throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, + "Suspiciously big size ({}) of all broken parts to remove.", + formatReadableSizeWithBinarySuffix(suspicious_broken_parts_bytes)); +} + + +void MergeTreeData::loadDataPartsFromWAL( + DataPartsVector & /* broken_parts_to_detach */, + DataPartsVector & duplicate_parts_to_remove, + MutableDataPartsVector & parts_from_wal, + DataPartsLock & part_lock) +{ + for (auto & part : parts_from_wal) + { + if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock)) + continue; + + part->modification_time = time(nullptr); + /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later + part->setState(DataPartState::Committed); + + auto [it, inserted] = data_parts_indexes.insert(part); + if (!inserted) + { + if ((*it)->checksums.getTotalChecksumHex() == part->checksums.getTotalChecksumHex()) + { + LOG_ERROR(log, "Remove duplicate part {}", part->getFullPath()); + duplicate_parts_to_remove.push_back(part); + } + else + throw Exception("Part " + part->name + " already exists but with different checksums", ErrorCodes::DUPLICATE_DATA_PART); + } + + addPartContributionToDataVolume(part); + } +} + void MergeTreeData::loadDataParts(bool skip_sanity_checks) { @@ -904,7 +1159,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) auto metadata_snapshot = getInMemoryMetadataPtr(); const auto settings = getSettings(); - std::vector> part_names_with_disks; MutableDataPartsVector parts_from_wal; Strings part_file_names; @@ -934,193 +1188,90 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } - /// Reversed order to load part from low priority disks firstly. - /// Used for keep part on low priority disk if duplication found - for (auto disk_it = disks.rbegin(); disk_it != disks.rend(); ++disk_it) + /// Collect part names by disk. + std::map>> disk_part_map; + std::map disk_wal_part_map; + ThreadPool pool(disks.size()); + std::mutex wal_init_lock; + for (const auto & disk_ptr : disks) { - auto disk_ptr = *disk_it; + auto & disk_parts = disk_part_map[disk_ptr->getName()]; + auto & disk_wal_parts = disk_wal_part_map[disk_ptr->getName()]; - for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) + pool.scheduleOrThrowOnError([&, disk_ptr]() { - /// Skip temporary directories, file 'format_version.txt' and directory 'detached'. - if (startsWith(it->name(), "tmp") - || it->name() == MergeTreeData::FORMAT_VERSION_FILE_NAME - || it->name() == MergeTreeData::DETACHED_DIR_NAME) - continue; - - if (!startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) - part_names_with_disks.emplace_back(it->name(), disk_ptr); - else if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) + for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) { - /// Create and correctly initialize global WAL object - write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); - for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext())) - parts_from_wal.push_back(std::move(part)); + /// Skip temporary directories, file 'format_version.txt' and directory 'detached'. + if (startsWith(it->name(), "tmp") || it->name() == MergeTreeData::FORMAT_VERSION_FILE_NAME + || it->name() == MergeTreeData::DETACHED_DIR_NAME) + continue; + + if (!startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) + disk_parts.emplace_back(std::make_pair(it->name(), disk_ptr)); + else if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) + { + std::unique_lock lock(wal_init_lock); + if (write_ahead_log != nullptr) + throw Exception( + "There are multiple WAL files appeared in current storage policy. You need to resolve this manually", + ErrorCodes::CORRUPTED_DATA); + + write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); + for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext())) + disk_wal_parts.push_back(std::move(part)); + } + else if (settings->in_memory_parts_enable_wal) + { + MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); + for (auto && part : wal.restore(metadata_snapshot, getContext())) + disk_wal_parts.push_back(std::move(part)); + } } - else if (settings->in_memory_parts_enable_wal) - { - MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); - for (auto && part : wal.restore(metadata_snapshot, getContext())) - parts_from_wal.push_back(std::move(part)); - } - } - } - - auto part_lock = lockParts(); - data_parts_indexes.clear(); - - if (part_names_with_disks.empty() && parts_from_wal.empty()) - { - LOG_DEBUG(log, "There are no data parts"); - return; - } - - /// Parallel loading of data parts. - size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_names_with_disks.size()); - - std::mutex mutex; - - DataPartsVector broken_parts_to_detach; - size_t suspicious_broken_parts = 0; - size_t suspicious_broken_parts_bytes = 0; - - std::atomic has_adaptive_parts = false; - std::atomic has_non_adaptive_parts = false; - - ThreadPool pool(num_threads); - - for (auto & part_names_with_disk : part_names_with_disks) - { - pool.scheduleOrThrowOnError([&] - { - const auto & [part_name, part_disk_ptr] = part_names_with_disk; - - auto part_opt = MergeTreePartInfo::tryParsePartName(part_name, format_version); - - if (!part_opt) - return; - - auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); - auto part = createPart(part_name, *part_opt, single_disk_volume, part_name); - bool broken = false; - - String part_path = fs::path(relative_data_path) / part_name; - String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; - - if (part_disk_ptr->exists(marker_path)) - { - /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist - size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); - LOG_WARNING(log, - "Detaching stale part {}{} (size: {}), which should have been deleted after a move. " - "That can only happen after unclean restart of ClickHouse after move of a part having an operation blocking that stale copy of part.", - getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); - std::lock_guard loading_lock(mutex); - - broken_parts_to_detach.push_back(part); - - ++suspicious_broken_parts; - suspicious_broken_parts_bytes += size_of_part; - - return; - } - - try - { - part->loadColumnsChecksumsIndexes(require_part_metadata, true); - } - catch (const Exception & e) - { - /// Don't count the part as broken if there is not enough memory to load it. - /// In fact, there can be many similar situations. - /// But it is OK, because there is a safety guard against deleting too many parts. - if (isNotEnoughMemoryErrorCode(e.code())) - throw; - - broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - catch (...) - { - broken = true; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - /// Ignore broken parts that can appear as a result of hard server restart. - if (broken) - { - /// NOTE: getBytesOnDisk() cannot be used here, since it maybe zero of checksums.txt will not exist - size_t size_of_part = IMergeTreeDataPart::calculateTotalSizeOnDisk(part->volume->getDisk(), part->getFullRelativePath()); - - LOG_ERROR(log, - "Detaching broken part {}{} (size: {}). " - "If it happened after update, it is likely because of backward incompability. " - "You need to resolve this manually", - getFullPathOnDisk(part_disk_ptr), part_name, formatReadableSizeWithBinarySuffix(size_of_part)); - std::lock_guard loading_lock(mutex); - - broken_parts_to_detach.push_back(part); - - ++suspicious_broken_parts; - suspicious_broken_parts_bytes += size_of_part; - - return; - } - - if (!part->index_granularity_info.is_adaptive) - has_non_adaptive_parts.store(true, std::memory_order_relaxed); - else - has_adaptive_parts.store(true, std::memory_order_relaxed); - - part->modification_time = part_disk_ptr->getLastModified(fs::path(relative_data_path) / part_name).epochTime(); - - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->setState(DataPartState::Committed); - - std::lock_guard loading_lock(mutex); - - if (!data_parts_indexes.insert(part).second) - throw Exception(ErrorCodes::DUPLICATE_DATA_PART, "Part {} already exists", part->name); - - addPartContributionToDataVolume(part); }); } pool.wait(); - for (auto & part : parts_from_wal) + for (auto & [_, disk_wal_parts] : disk_wal_part_map) + parts_from_wal.insert( + parts_from_wal.end(), std::make_move_iterator(disk_wal_parts.begin()), std::make_move_iterator(disk_wal_parts.end())); + + size_t num_parts = 0; + std::queue>> parts_queue; + for (auto & [_, disk_parts] : disk_part_map) { - if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock)) + if (disk_parts.empty()) continue; - - part->modification_time = time(nullptr); - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->setState(DataPartState::Committed); - - if (!data_parts_indexes.insert(part).second) - throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); - - addPartContributionToDataVolume(part); + num_parts += disk_parts.size(); + parts_queue.push(std::move(disk_parts)); } - if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) - throw Exception("Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", ErrorCodes::LOGICAL_ERROR); + auto part_lock = lockParts(); + data_parts_indexes.clear(); - has_non_adaptive_index_granularity_parts = has_non_adaptive_parts; + if (num_parts == 0 && parts_from_wal.empty()) + { + LOG_DEBUG(log, "There are no data parts"); + return; + } - if (suspicious_broken_parts > settings->max_suspicious_broken_parts && !skip_sanity_checks) - throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, - "Suspiciously many ({}) broken parts to remove.", - suspicious_broken_parts); - if (suspicious_broken_parts_bytes > settings->max_suspicious_broken_parts_bytes && !skip_sanity_checks) - throw Exception(ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS, - "Suspiciously big size ({}) of all broken parts to remove.", - formatReadableSizeWithBinarySuffix(suspicious_broken_parts_bytes)); + DataPartsVector broken_parts_to_detach; + DataPartsVector duplicate_parts_to_remove; + + if (num_parts > 0) + loadDataPartsFromDisk( + broken_parts_to_detach, duplicate_parts_to_remove, pool, num_parts, parts_queue, skip_sanity_checks, settings); + + if (!parts_from_wal.empty()) + loadDataPartsFromWAL(broken_parts_to_detach, duplicate_parts_to_remove, parts_from_wal, part_lock); for (auto & part : broken_parts_to_detach) part->renameToDetached("broken-on-start"); /// detached parts must not have '_' in prefixes + for (auto & part : duplicate_parts_to_remove) + part->remove(); /// Delete from the set of current parts those parts that are covered by another part (those parts that /// were merged), but that for some reason are still not deleted from the filesystem. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e7f1db8f3ec..b87a756bf9f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -1143,6 +1143,21 @@ private: /// Returns default settings for storage with possible changes from global config. virtual std::unique_ptr getDefaultSettings() const = 0; + + void loadDataPartsFromDisk( + DataPartsVector & broken_parts_to_detach, + DataPartsVector & duplicate_parts_to_remove, + ThreadPool & pool, + size_t num_parts, + std::queue>> & parts_queue, + bool skip_sanity_checks, + const MergeTreeSettingsPtr & settings); + + void loadDataPartsFromWAL( + DataPartsVector & broken_parts_to_detach, + DataPartsVector & duplicate_parts_to_remove, + MutableDataPartsVector & parts_from_wal, + DataPartsLock & part_lock); }; /// RAII struct to record big parts that are submerging or emerging. From 74e1900034e3beec14fdcf67c9f31f79f271ae2d Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 05:23:00 +0000 Subject: [PATCH 561/950] fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 14 +++++++++---- .../FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 20 +++++++++---------- 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 4f2a775c265..076475b8635 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -24,14 +24,14 @@ DirectoryWatcherBase::DirectoryWatcherBase( : WithContext(context_->getGlobalContext()), owner(owner_), path(path_), event_mask(event_mask_) { if (!std::filesystem::exists(path)) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist.", path); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist", path); if (!std::filesystem::is_directory(path)) - throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory.", path); + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory", path); fd = inotify_init(); if (fd == -1) - throw Exception("cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); + throw Exception("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); @@ -54,7 +54,7 @@ void DirectoryWatcherBase::watchFunc() int wd = inotify_add_watch(fd, path.c_str(), mask); if (wd == -1) { - owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed.", path)); + owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed", path)); } std::string buffer; @@ -62,11 +62,17 @@ void DirectoryWatcherBase::watchFunc() fd_set fds; while (!stopped) { +#if defined(__clang__) #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wreserved-identifier" FD_ZERO(&fds); #pragma clang diagnostic pop +#else + FD_ZERO(&fds); +#endif + + FD_SET(fd, &fds); struct timeval tv; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index cf52b2a2ccc..745d8c46a63 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -107,7 +107,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); if (!file_ctx.reader) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized.", file_meta.file_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized", file_meta.file_name); auto & reader = file_ctx.reader.value(); Record record; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c8ffd44369e..516be811666 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -270,7 +270,7 @@ void StorageFileLog::deserialize() { throw Exception( ErrorCodes::NOT_REGULAR_FILE, - "The file {} under {} is not a regular file when deserializing meta files.", + "The file {} under {} is not a regular file when deserializing meta files", dir_entry.path().c_str(), root_meta_path); } @@ -281,12 +281,12 @@ void StorageFileLog::deserialize() if (!tryReadIntText(inode, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", dir_entry.path().c_str()); } assertChar('\n', in); if (!tryReadIntText(last_written_pos, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", dir_entry.path().c_str()); } meta.file_name = dir_entry.path().filename(); @@ -322,7 +322,7 @@ Pipe StorageFileLog::read( { throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, - "Can not read from table {}, because it has been depended by other tables.", + "Can not read from table {}, because it has been depended by other tables", table_id.getTableName()); } @@ -433,7 +433,7 @@ void StorageFileLog::assertStreamGood(const std::ifstream & reader) { if (!reader.good()) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Stream is in bad state."); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Stream is in bad state"); } } @@ -457,7 +457,7 @@ void StorageFileLog::openFilesAndSetPos() auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken.", file); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken", file); } /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; @@ -516,16 +516,16 @@ void StorageFileLog::checkOffsetIsValid(const String & full_name, UInt64 offset) if (!tryReadIntText(_, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", full_name); } assertChar('\n', in); if (!tryReadIntText(last_written_pos, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", full_name); } if (last_written_pos > offset) throw Exception( - ErrorCodes::LOGICAL_ERROR, "Last stored last_written_pos in meta file {} is bigger than current last_written_pos.", full_name); + ErrorCodes::LOGICAL_ERROR, "Last stored last_written_pos in meta file {} is bigger than current last_written_pos", full_name); } size_t StorageFileLog::getMaxBlockSize() const @@ -633,7 +633,7 @@ bool StorageFileLog::streamToViews() auto table_id = getStorageID(); auto table = DatabaseCatalog::instance().getTable(table_id, getContext()); if (!table) - throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); auto max_streams_number = std::min(filelog_settings->max_threads.value, file_infos.file_names.size()); From 1623bfa1a27a8175f964197194dd9d3d9bf7c84c Mon Sep 17 00:00:00 2001 From: Vitaly Orlov Date: Wed, 13 Oct 2021 07:33:53 +0300 Subject: [PATCH 562/950] review fixes --- src/Functions/FunctionsJSON.h | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 4f91440105d..4fd4e6b2d00 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -1362,7 +1362,7 @@ public: static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { - return std::make_unique(); + return std::make_unique(std::make_shared()); } static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } @@ -1374,16 +1374,15 @@ public: auto object = element.getObject(); - auto & col_arr = assert_cast(dest); - auto & col_tuple = assert_cast(col_arr.getData()); - auto & col_key = assert_cast(col_tuple.getColumn(0)); + ColumnArray & col_res = assert_cast(dest); + auto & col_key = assert_cast(col_res.getData()); for (const auto & [key, value] : object) { col_key.insertData(key.data(), key.size()); } - col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); + col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); return true; } }; From 9bc7ca6dd5d233f5f5e82efd27de4be38b900271 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 13 Oct 2021 09:56:30 +0300 Subject: [PATCH 563/950] Update hero.html --- website/templates/index/hero.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/templates/index/hero.html b/website/templates/index/hero.html index 79ef8974ca1..795a89a44f2 100644 --- a/website/templates/index/hero.html +++ b/website/templates/index/hero.html @@ -33,11 +33,11 @@

    21 October 2021
    EMEA, Americas
    - 4:00pm UTC / 9am PST + 4:00pm UTC / 9:00am PST

    From a140f4127e67d122f18e2909387b684e959e8d3d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 13 Oct 2021 09:56:34 +0300 Subject: [PATCH 564/950] Fix ExternalLoader without additional mutex. --- src/Interpreters/ExternalLoader.cpp | 66 ++++++++++++++--------------- src/Interpreters/ExternalLoader.h | 8 ---- 2 files changed, 33 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index dc8466f3c26..b2cd9495feb 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -133,7 +133,13 @@ public: settings = settings_; } - using ObjectConfigsPtr = std::shared_ptr>>; + struct ObjectConfigs + { + std::unordered_map> configs_by_name; + size_t counter = 0; + }; + + using ObjectConfigsPtr = std::shared_ptr; /// Reads all repositories. ObjectConfigsPtr read() @@ -336,7 +342,7 @@ private: need_collect_object_configs = false; // Generate new result. - auto new_configs = std::make_shared>>(); + auto new_configs = std::make_shared(); for (const auto & [repository, repository_info] : repositories) { @@ -344,8 +350,8 @@ private: { for (const auto & [object_name, key_in_config] : file_info.objects) { - auto already_added_it = new_configs->find(object_name); - if (already_added_it == new_configs->end()) + auto already_added_it = new_configs->configs_by_name.find(object_name); + if (already_added_it == new_configs->configs_by_name.end()) { auto new_config = std::make_shared(); new_config->config = file_info.file_contents; @@ -353,7 +359,7 @@ private: new_config->repository_name = repository->getName(); new_config->from_temp_repository = repository->isTemporary(); new_config->path = path; - new_configs->emplace(object_name, std::move(new_config)); + new_configs->configs_by_name.emplace(object_name, std::move(new_config)); } else { @@ -372,6 +378,7 @@ private: } } + new_configs->counter = counter++; object_configs = new_configs; } @@ -383,6 +390,7 @@ private: std::unordered_map repositories; ObjectConfigsPtr object_configs; bool need_collect_object_configs = false; + size_t counter = 0; }; @@ -433,13 +441,22 @@ public: if (configs == new_configs) return; + /// The following check prevents a race when two threads are trying to update configuration + /// at almost the same time: + /// 1) first thread reads a configuration (for example as a part of periodic updates) + /// 2) second thread sets a new configuration (for example after executing CREATE DICTIONARY) + /// 3) first thread sets the configuration it read in 1) and thus discards the changes made in 2). + /// So we use `counter` here to ensure we exchange the current configuration only for a newer one. + if (configs && (configs->counter >= new_configs->counter)) + return; + configs = new_configs; std::vector removed_names; for (auto & [name, info] : infos) { - auto new_config_it = new_configs->find(name); - if (new_config_it == new_configs->end()) + auto new_config_it = new_configs->configs_by_name.find(name); + if (new_config_it == new_configs->configs_by_name.end()) { removed_names.emplace_back(name); } @@ -462,7 +479,7 @@ public: } /// Insert to the map those objects which added to the new configuration. - for (const auto & [name, config] : *new_configs) + for (const auto & [name, config] : new_configs->configs_by_name) { if (infos.find(name) == infos.end()) { @@ -1198,8 +1215,8 @@ class ExternalLoader::PeriodicUpdater : private boost::noncopyable public: static constexpr UInt64 check_period_sec = 5; - PeriodicUpdater(LoadablesConfigReader & config_files_reader_, LoadingDispatcher & loading_dispatcher_, std::recursive_mutex & config_mutex_) - : config_files_reader(config_files_reader_), loading_dispatcher(loading_dispatcher_), config_mutex(config_mutex_) + PeriodicUpdater(LoadablesConfigReader & config_files_reader_, LoadingDispatcher & loading_dispatcher_) + : config_files_reader(config_files_reader_), loading_dispatcher(loading_dispatcher_) { } @@ -1242,21 +1259,14 @@ private: while (!event.wait_for(lock, std::chrono::seconds(check_period_sec), pred)) { lock.unlock(); - { - { - std::lock_guard config_lock{config_mutex}; - loading_dispatcher.setConfiguration(config_files_reader.read()); - } - loading_dispatcher.reloadOutdated(); - } + loading_dispatcher.setConfiguration(config_files_reader.read()); + loading_dispatcher.reloadOutdated(); lock.lock(); } } LoadablesConfigReader & config_files_reader; LoadingDispatcher & loading_dispatcher; - - std::recursive_mutex & config_mutex; mutable std::mutex mutex; bool enabled = false; ThreadFromGlobalPool thread; @@ -1270,7 +1280,7 @@ ExternalLoader::ExternalLoader(const String & type_name_, Poco::Logger * log_) [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, type_name_, log_)) - , periodic_updater(std::make_unique(*config_files_reader, *loading_dispatcher, config_mutex)) + , periodic_updater(std::make_unique(*config_files_reader, *loading_dispatcher)) , type_name(type_name_) , log(log_) { @@ -1282,13 +1292,12 @@ scope_guard ExternalLoader::addConfigRepository(std::unique_ptrgetName(); - std::lock_guard lock{config_mutex}; + config_files_reader->addConfigRepository(std::move(repository)); reloadConfig(name); return [this, ptr, name]() { - std::lock_guard config_lock{config_mutex}; config_files_reader->removeConfigRepository(ptr); reloadConfig(name); }; @@ -1387,10 +1396,7 @@ ReturnType ExternalLoader::load(const FilterByNameFunction & filter) const template ReturnType ExternalLoader::loadOrReload(const String & name) const { - { - std::lock_guard lock{config_mutex}; - loading_dispatcher->setConfiguration(config_files_reader->read()); - } + loading_dispatcher->setConfiguration(config_files_reader->read()); auto result = loading_dispatcher->tryLoadOrReload(name, WAIT); checkLoaded(result, true); return convertTo(result); @@ -1399,10 +1405,7 @@ ReturnType ExternalLoader::loadOrReload(const String & name) const template ReturnType ExternalLoader::loadOrReload(const FilterByNameFunction & filter) const { - { - std::lock_guard lock{config_mutex}; - loading_dispatcher->setConfiguration(config_files_reader->read()); - } + loading_dispatcher->setConfiguration(config_files_reader->read()); auto results = loading_dispatcher->tryLoadOrReload(filter, WAIT); checkLoaded(results, true); return convertTo(results); @@ -1490,19 +1493,16 @@ void ExternalLoader::checkLoaded(const ExternalLoader::LoadResults & results, void ExternalLoader::reloadConfig() const { - std::lock_guard lock{config_mutex}; loading_dispatcher->setConfiguration(config_files_reader->read()); } void ExternalLoader::reloadConfig(const String & repository_name) const { - std::lock_guard lock{config_mutex}; loading_dispatcher->setConfiguration(config_files_reader->read(repository_name)); } void ExternalLoader::reloadConfig(const String & repository_name, const String & path) const { - std::lock_guard lock{config_mutex}; loading_dispatcher->setConfiguration(config_files_reader->read(repository_name, path)); } diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index ed17138754a..81e043e40bf 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -219,14 +219,6 @@ private: LoadablePtr createObject(const String & name, const ObjectConfig & config, const LoadablePtr & previous_version) const; - /// We have to read configuration from LoadablesConfigReader and load configuration using LoadingDispatcher atomically. - /// Otherwise we can read configuration in one thread, then read and load newer configuration in another thread, - /// and then load outdated configuration from the first thread. - /// Remarkably, each class (LoadablesConfigReader, LoadingDispatcher, PeriodicUpdater) has own mutex for own purposes, - /// but it does not save from complex logical race conditions. - /// TODO Refactor dictionaries loading and get rid of this. - mutable std::recursive_mutex config_mutex; - class LoadablesConfigReader; std::unique_ptr config_files_reader; From 038ca829e4bd7f3720a967695ea1aa0d9dc4ed48 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 12:07:58 +0300 Subject: [PATCH 565/950] Add test with GLOBAL IN and totals. --- .../queries/0_stateless/02096_totals_global_in_bug.reference | 4 ++++ tests/queries/0_stateless/02096_totals_global_in_bug.sql | 2 ++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02096_totals_global_in_bug.reference create mode 100644 tests/queries/0_stateless/02096_totals_global_in_bug.sql diff --git a/tests/queries/0_stateless/02096_totals_global_in_bug.reference b/tests/queries/0_stateless/02096_totals_global_in_bug.reference new file mode 100644 index 00000000000..a536e1a5329 --- /dev/null +++ b/tests/queries/0_stateless/02096_totals_global_in_bug.reference @@ -0,0 +1,4 @@ +0 +2 + +2 diff --git a/tests/queries/0_stateless/02096_totals_global_in_bug.sql b/tests/queries/0_stateless/02096_totals_global_in_bug.sql new file mode 100644 index 00000000000..ac4f2b9d2ba --- /dev/null +++ b/tests/queries/0_stateless/02096_totals_global_in_bug.sql @@ -0,0 +1,2 @@ +select sum(number) from remote('127.0.0.{2,3}', numbers(2)) where number global in (select sum(number) from numbers(2) group by number with totals) group by number with totals + From 868dbc4bad1a90dc428bb9f7a6e81a5d34a27259 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 09:10:41 +0000 Subject: [PATCH 566/950] use poll replace select fix fix --- CMakeLists.txt | 3 +-- cmake/find/filelog.cmake | 13 ++++++++++ src/Storages/FileLog/DirectoryWatcherBase.cpp | 25 ++++--------------- 3 files changed, 19 insertions(+), 22 deletions(-) create mode 100644 cmake/find/filelog.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index f6cbe89eafc..7e2ace06397 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -570,6 +570,7 @@ include (cmake/find/yaml-cpp.cmake) include (cmake/find/s2geometry.cmake) include (cmake/find/nlp.cmake) include (cmake/find/bzip2.cmake) +include (cmake/find/filelog.cmake) if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") @@ -583,8 +584,6 @@ include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) -set (USE_FILELOG ON) - set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/cmake/find/filelog.cmake b/cmake/find/filelog.cmake new file mode 100644 index 00000000000..6c38ba33fbf --- /dev/null +++ b/cmake/find/filelog.cmake @@ -0,0 +1,13 @@ +option (ENABLE_FILELOG "Enable FILELOG" ON) + +if (NOT ENABLE_FILELOG) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use StorageFileLog with ENABLE_FILELOG=OFF") + return() +endif() + +# StorageFileLog only support Linux platform +if(${CMAKE_SYSTEM_NAME} MATCHES "Linux") + set (USE_FILELOG 1) + message (STATUS "Using StorageFileLog = 1") +endif(${CMAKE_SYSTEM_NAME} MATCHES "Linux") + diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 076475b8635..4f6e429e1fd 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB { @@ -59,27 +59,12 @@ void DirectoryWatcherBase::watchFunc() std::string buffer; buffer.resize(buffer_size); - fd_set fds; + pollfd pfd; + pfd.fd = fd; + pfd.events = POLLIN; while (!stopped) { -#if defined(__clang__) -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wreserved-identifier" - FD_ZERO(&fds); -#pragma clang diagnostic pop - -#else - FD_ZERO(&fds); -#endif - - - FD_SET(fd, &fds); - - struct timeval tv; - tv.tv_sec = 0; - tv.tv_usec = 200000; - - if (select(fd + 1, &fds, nullptr, nullptr, &tv) == 1) + if (poll(&pfd, 1, 5000) > 0 && pfd.revents & POLLIN) { int n = read(fd, buffer.data(), buffer.size()); int i = 0; From 5dc204c7168b75902121d102623767db000515fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9E=A2=E6=9C=A8?= Date: Wed, 13 Oct 2021 17:44:07 +0800 Subject: [PATCH 567/950] update adopters to update Alibaba Cloud. --- docs/en/introduction/adopters.md | 1 + docs/zh/introduction/adopters.md | 2 ++ 2 files changed, 3 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index c511bd97a7c..3e13d56e252 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -15,6 +15,7 @@ toc_title: Adopters | AdScribe | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) | | Ahrefs | SEO | Analytics | — | — | [Job listing](https://ahrefs.com/jobs/data-scientist-search) | | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | +| Alibaba Cloud | Cloud | E-MapReduce | — | — | [Official Website](https://help.aliyun.com/document_detail/212195.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.com/meetup22/aloha.pdf) | | Altinity | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | diff --git a/docs/zh/introduction/adopters.md b/docs/zh/introduction/adopters.md index e67e4bfca6e..66341b1265d 100644 --- a/docs/zh/introduction/adopters.md +++ b/docs/zh/introduction/adopters.md @@ -11,6 +11,8 @@ toc_title: "ClickHouse用户" | 公司简介 | 行业 | 用例 | 群集大小 | (Un)压缩数据大小\* | 参考资料 | |-----------------------------------------------------------------|------------------|----------------|---------------------------------------------------|----------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | [2gis](https://2gis.ru) | 地图 | 监测 | — | — | [俄文,2019年7月](https://youtu.be/58sPkXfq6nw) | +| [阿里云](https://cn.aliyun.com/) | 公有云 | 全托管服务 | — | — | [官方网站](https://help.aliyun.com/product/144466.html) | +| [阿里云](https://cn.aliyun.com/) | 公有云 | E-MapReduce | — | — | [官方网站](https://help.aliyun.com/document_detail/212195.html) | | [Aloha 浏览器](https://alohabrowser.com/) | 移动应用程序 | 浏览器后端 | — | — | [俄文幻灯片,2019年5月](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/aloha.pdf) | | [阿玛迪斯](https://amadeus.com/) | 旅行 | 分析 | — | — | [新闻稿,四月2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | | [Appsflyer](https://www.appsflyer.com) | 移动分析 | 主要产品 | — | — | [俄文,2019年7月](https://www.youtube.com/watch?v=M3wbRlcpBbY) | From 6b95b706f155b847eaca2c34b5413d6e9376d764 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 12:49:59 +0300 Subject: [PATCH 568/950] Update programs/odbc-bridge/ODBCBlockInputStream.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- programs/odbc-bridge/ODBCBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 110df30060d..7278b62cfcb 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes ODBCSource::ODBCSource( nanodbc::ConnectionHolderPtr connection_holder, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_) : ISource(sample_block) - , log(&Poco::Logger::get("ODBCBlockInputStream")) + , log(&Poco::Logger::get("ODBCSource")) , max_block_size{max_block_size_} , query(query_str) { From b5bc3853910e4ed56ccf9f0f68562893619eee02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 12:50:20 +0300 Subject: [PATCH 569/950] Update programs/odbc-bridge/ODBCBlockOutputStream.cpp Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- programs/odbc-bridge/ODBCBlockOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index 0e6770c1295..710614130c3 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -24,7 +24,7 @@ ODBCSink::ODBCSink( ContextPtr local_context_, IdentifierQuotingStyle quoting_) : ISink(sample_block_) - , log(&Poco::Logger::get("ODBCBlockOutputStream")) + , log(&Poco::Logger::get("ODBCSink")) , connection_holder(std::move(connection_holder_)) , db_name(remote_database_name_) , table_name(remote_table_name_) From a5fa5c7ea3ece49e282bec06654831f2060553c3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 13:01:08 +0300 Subject: [PATCH 570/950] Move formats to Impl --- src/Client/ClientBase.cpp | 10 ++++---- src/Formats/NullFormat.cpp | 23 ------------------- .../Formats/Impl}/NativeFormat.cpp | 0 src/Processors/Formats/Impl/NullFormat.cpp | 18 ++++----------- .../Formats/Impl}/NullFormat.h | 0 5 files changed, 10 insertions(+), 41 deletions(-) delete mode 100644 src/Formats/NullFormat.cpp rename src/{Formats => Processors/Formats/Impl}/NativeFormat.cpp (100%) rename src/{Formats => Processors/Formats/Impl}/NullFormat.h (100%) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 81e9cc47134..e260ed11bd4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -38,7 +38,7 @@ #include #include -#include +#include #include #include #include @@ -359,10 +359,10 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) current_format = "Vertical"; /// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly. - // if (!need_render_progress) - // output_format = global_context->getOutputFormatParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); - // else - output_format = global_context->getOutputFormat(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + if (!need_render_progress) + output_format = global_context->getOutputFormatParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + else + output_format = global_context->getOutputFormat(current_format, out_file_buf ? *out_file_buf : *out_buf, block); output_format->doWritePrefix(); } diff --git a/src/Formats/NullFormat.cpp b/src/Formats/NullFormat.cpp deleted file mode 100644 index 8613293cb13..00000000000 --- a/src/Formats/NullFormat.cpp +++ /dev/null @@ -1,23 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -WriteBuffer NullOutputFormat::empty_buffer(nullptr, 0); - -void registerOutputFormatNull(FormatFactory & factory) -{ - factory.registerOutputFormat("Null", []( - WriteBuffer &, - const Block & sample, - const RowOutputFormatParams &, - const FormatSettings &) - { - return std::make_shared(sample); - }); -} - -} diff --git a/src/Formats/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp similarity index 100% rename from src/Formats/NativeFormat.cpp rename to src/Processors/Formats/Impl/NativeFormat.cpp diff --git a/src/Processors/Formats/Impl/NullFormat.cpp b/src/Processors/Formats/Impl/NullFormat.cpp index a22005a1c12..6c457c71d14 100644 --- a/src/Processors/Formats/Impl/NullFormat.cpp +++ b/src/Processors/Formats/Impl/NullFormat.cpp @@ -1,30 +1,22 @@ -#include +#include #include +#include namespace DB { -class NullOutputFormat : public IOutputFormat -{ -public: - NullOutputFormat(const Block & header, WriteBuffer & out_) : IOutputFormat(header, out_) {} - - String getName() const override { return "NullOutputFormat"; } - -protected: - void consume(Chunk) override {} -}; +WriteBuffer NullOutputFormat::empty_buffer(nullptr, 0); void registerOutputFormatNull(FormatFactory & factory) { factory.registerOutputFormat("Null", []( - WriteBuffer & buf, + WriteBuffer &, const Block & sample, const RowOutputFormatParams &, const FormatSettings &) { - return std::make_shared(sample, buf); + return std::make_shared(sample); }); } diff --git a/src/Formats/NullFormat.h b/src/Processors/Formats/Impl/NullFormat.h similarity index 100% rename from src/Formats/NullFormat.h rename to src/Processors/Formats/Impl/NullFormat.h From a69b0d9818d1bb9f934ff76d144e3fa5ea6390ad Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Oct 2021 13:21:07 +0300 Subject: [PATCH 571/950] Follow-up to #29901 (#30003) * more informative logs * Update ZooKeeper.cpp * fix --- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Common/ZooKeeper/TestKeeper.cpp | 8 ++++---- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 18 +++++++++--------- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 +++--- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- src/Interpreters/Context.cpp | 2 +- 8 files changed, 21 insertions(+), 21 deletions(-) diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index 1a816616c7c..65ebd8ee615 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -481,7 +481,7 @@ public: MultiCallback callback) = 0; /// Expire session and finish all pending requests - virtual void finalize() = 0; + virtual void finalize(const String & reason) = 0; }; } diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index be210644e70..065b1cf65ba 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -493,7 +493,7 @@ TestKeeper::~TestKeeper() { try { - finalize(); + finalize(__PRETTY_FUNCTION__); if (processing_thread.joinable()) processing_thread.join(); } @@ -556,12 +556,12 @@ void TestKeeper::processingThread() catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); - finalize(); + finalize(__PRETTY_FUNCTION__); } } -void TestKeeper::finalize() +void TestKeeper::finalize(const String &) { { std::lock_guard lock(push_request_mutex); @@ -661,7 +661,7 @@ void TestKeeper::pushRequest(RequestInfo && request) } catch (...) { - finalize(); + finalize(__PRETTY_FUNCTION__); throw; } } diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index b46f98c0074..e57471341e8 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -83,7 +83,7 @@ public: const Requests & requests, MultiCallback callback) override; - void finalize() override; + void finalize(const String & reason) override; struct Node { diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index e347e6de362..3d505c088db 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -269,7 +269,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::List), path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -330,7 +330,7 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -400,7 +400,7 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Remove), path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -432,7 +432,7 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Exists), path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -466,7 +466,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Get), path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -539,7 +539,7 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Set), path)); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -591,7 +591,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests if (future_result.wait_for(std::chrono::milliseconds(operation_timeout_ms)) != std::future_status::ready) { - impl->finalize(); + impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Multi), requests[0]->getPath())); return Coordination::Error::ZOPERATIONTIMEOUT; } else @@ -1038,9 +1038,9 @@ Coordination::Error ZooKeeper::tryMultiNoThrow(const Coordination::Requests & re } } -void ZooKeeper::finalize() +void ZooKeeper::finalize(const String & reason) { - impl->finalize(); + impl->finalize(reason); } void ZooKeeper::setZooKeeperLog(std::shared_ptr zk_log_) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 5ee25aba1fb..8e015b1f331 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -274,7 +274,7 @@ public: /// * The node doesn't exist FutureGet asyncTryGet(const std::string & path); - void finalize(); + void finalize(const String & reason); void setZooKeeperLog(std::shared_ptr zk_log_); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index cf607a3d70e..59ed906db15 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -289,7 +289,7 @@ ZooKeeper::~ZooKeeper() { try { - finalize(false, false, "destructor called"); + finalize(false, false, "Destructor called"); if (send_thread.joinable()) send_thread.join(); @@ -610,7 +610,7 @@ void ZooKeeper::sendThread() catch (...) { tryLogCurrentException(log); - finalize(true, false, "exception in sendThread"); + finalize(true, false, "Exception in sendThread"); } } @@ -669,7 +669,7 @@ void ZooKeeper::receiveThread() catch (...) { tryLogCurrentException(log); - finalize(false, true, "exception in receiveThread"); + finalize(false, true, "Exception in receiveThread"); } } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 53908e5b0c7..74c0148e7b6 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -187,7 +187,7 @@ public: /// it will do read in another session, that read may not see the /// already performed write. - void finalize() override { finalize(false, false, "unknown"); } + void finalize(const String & reason) override { finalize(false, false, reason); } void setZooKeeperLog(std::shared_ptr zk_log_); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1a95a642e18..e4c6de8853b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1853,7 +1853,7 @@ static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const if (!zk || zk->configChanged(*config, config_name)) { if (zk) - zk->finalize(); + zk->finalize("Config changed"); zk = std::make_shared(*config, config_name, std::move(zk_log)); } From 7742b96497bfff9be35c7463bd21efce839c919b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Oct 2021 13:34:18 +0300 Subject: [PATCH 572/950] Remove metadata leftovers on drop database (#30054) * remove metadata leftovers on drop database * Update InterpreterDropQuery.cpp * Update DatabaseCatalog.cpp --- src/Core/Settings.h | 2 ++ src/Databases/DatabaseFactory.cpp | 1 + src/Databases/DatabaseOnDisk.cpp | 25 +++++++++++++++++-- src/Interpreters/DatabaseCatalog.cpp | 10 +++++--- src/Interpreters/DatabaseCatalog.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Interpreters/loadMetadata.cpp | 1 + .../test_restart_server/__init__.py | 0 tests/integration/test_restart_server/test.py | 22 ++++++++++++++++ .../0_stateless/01601_detach_permanently.sql | 4 +-- 11 files changed, 60 insertions(+), 11 deletions(-) create mode 100755 tests/integration/test_restart_server/__init__.py create mode 100755 tests/integration/test_restart_server/test.py diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6dcbf663dd5..a5767955045 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -523,6 +523,8 @@ class IColumn; M(Int64, remote_fs_read_backoff_threshold, 10000, "Max wait time when trying to read data for remote disk", 0) \ M(Int64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ \ + M(Bool, force_remove_data_recursively_on_drop, false, "Recursively remove data on DROP query. Avoids 'Directory not empty' error, but may silently remove detached data", 0) \ + \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 9cf600d5cdf..fc428aae9ac 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -71,6 +71,7 @@ DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & m /// Before 20.7 it's possible that .sql metadata file does not exist for some old database. /// In this case Ordinary database is created on server startup if the corresponding metadata directory exists. /// So we should remove metadata directory if database creation failed. + /// TODO remove this code created = fs::create_directory(metadata_path); DatabasePtr impl = getImpl(create, metadata_path, context); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index d681d4f83da..ea9635530c7 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -39,6 +39,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int TABLE_ALREADY_EXISTS; extern const int EMPTY_LIST_OF_COLUMNS_PASSED; + extern const int DATABASE_NOT_EMPTY; } @@ -544,8 +545,28 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const void DatabaseOnDisk::drop(ContextPtr local_context) { assert(tables.empty()); - fs::remove(local_context->getPath() + getDataPath()); - fs::remove(getMetadataPath()); + if (local_context->getSettingsRef().force_remove_data_recursively_on_drop) + { + fs::remove_all(local_context->getPath() + getDataPath()); + fs::remove_all(getMetadataPath()); + } + else + { + try + { + fs::remove(local_context->getPath() + getDataPath()); + fs::remove(getMetadataPath()); + } + catch (const fs::filesystem_error & e) + { + if (e.code() != std::errc::directory_not_empty) + throw Exception(Exception::CreateFromSTDTag{}, e); + throw Exception(ErrorCodes::DATABASE_NOT_EMPTY, "Cannot drop: {}. " + "Probably database contain some detached tables or metadata leftovers from Ordinary engine. " + "If you want to remove all data anyway, try to attach database back and drop it again " + "with enabled force_remove_data_recursively_on_drop setting", e.what()); + } + } } String DatabaseOnDisk::getObjectMetadataPath(const String & object_name) const diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index bff03be61a8..08cc260b536 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -329,7 +329,7 @@ void DatabaseCatalog::attachDatabase(const String & database_name, const Databas } -DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool drop, bool check_empty) +DatabasePtr DatabaseCatalog::detachDatabase(ContextPtr local_context, const String & database_name, bool drop, bool check_empty) { if (database_name == TEMPORARY_DATABASE) throw Exception("Cannot detach database with temporary tables.", ErrorCodes::DATABASE_ACCESS_DENIED); @@ -365,12 +365,14 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d if (drop) { /// Delete the database. - db->drop(getContext()); + db->drop(local_context); /// Old ClickHouse versions did not store database.sql files + /// Remove metadata dir (if exists) to avoid recreation of .sql file on server startup + fs::path database_metadata_dir = fs::path(getContext()->getPath()) / "metadata" / escapeForFileName(database_name); + fs::remove(database_metadata_dir); fs::path database_metadata_file = fs::path(getContext()->getPath()) / "metadata" / (escapeForFileName(database_name) + ".sql"); - if (fs::exists(database_metadata_file)) - fs::remove(database_metadata_file); + fs::remove(database_metadata_file); } return db; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index f2063e4199f..6079553b025 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -147,7 +147,7 @@ public: DatabasePtr getSystemDatabase() const; void attachDatabase(const String & database_name, const DatabasePtr & database); - DatabasePtr detachDatabase(const String & database_name, bool drop = false, bool check_empty = true); + DatabasePtr detachDatabase(ContextPtr local_context, const String & database_name, bool drop = false, bool check_empty = true); void updateDatabaseName(const String & old_name, const String & new_name); /// database_name must be not empty diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c098c6e0506..530b10703c5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -295,7 +295,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) assert(removed); } if (added) - DatabaseCatalog::instance().detachDatabase(database_name, false, false); + DatabaseCatalog::instance().detachDatabase(getContext(), database_name, false, false); throw; } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 509211df3b6..5394c1ecaf0 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -369,7 +369,7 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, database->assertCanBeDetached(true); /// DETACH or DROP database itself - DatabaseCatalog::instance().detachDatabase(database_name, drop, database->shouldBeEmptyOnDetach()); + DatabaseCatalog::instance().detachDatabase(getContext(), database_name, drop, database->shouldBeEmptyOnDetach()); } } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 858b4281f5a..6a3db48e835 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -72,6 +72,7 @@ static void loadDatabase( } else if (fs::exists(fs::path(database_path))) { + /// TODO Remove this code (it's required for compatibility with versions older than 20.7) /// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default) database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary"; } diff --git a/tests/integration/test_restart_server/__init__.py b/tests/integration/test_restart_server/__init__.py new file mode 100755 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_restart_server/test.py b/tests/integration/test_restart_server/test.py new file mode 100755 index 00000000000..47797f7c4a5 --- /dev/null +++ b/tests/integration/test_restart_server/test.py @@ -0,0 +1,22 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', stay_alive=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_drop_memory_database(start_cluster): + node.query("CREATE DATABASE test ENGINE Memory") + node.query("CREATE TABLE test.test_table(a String) ENGINE Memory") + node.query("DROP DATABASE test") + node.restart_clickhouse(kill=True) + assert node.query("SHOW DATABASES LIKE 'test'").strip() == "" + diff --git a/tests/queries/0_stateless/01601_detach_permanently.sql b/tests/queries/0_stateless/01601_detach_permanently.sql index 3bf05b872b9..97797a59af5 100644 --- a/tests/queries/0_stateless/01601_detach_permanently.sql +++ b/tests/queries/0_stateless/01601_detach_permanently.sql @@ -131,7 +131,7 @@ SELECT 'And detach permanently again to check how database drop will behave'; DETACH table test1601_detach_permanently_ordinary.test_name_reuse PERMANENTLY; SELECT 'DROP database - Directory not empty error, but database detached'; -DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 1001 } +DROP DATABASE test1601_detach_permanently_ordinary; -- { serverError 219 } ATTACH DATABASE test1601_detach_permanently_ordinary; @@ -205,7 +205,7 @@ SELECT 'And detach permanently again to check how database drop will behave'; DETACH table test1601_detach_permanently_lazy.test_name_reuse PERMANENTLY; SELECT 'DROP database - Directory not empty error, but database deteched'; -DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 1001 } +DROP DATABASE test1601_detach_permanently_lazy; -- { serverError 219 } ATTACH DATABASE test1601_detach_permanently_lazy; From 923c76fc9914d66593fd69d93684db2bf5182ab3 Mon Sep 17 00:00:00 2001 From: zhangxiao871 <821008736@qq.com> Date: Wed, 13 Oct 2021 18:38:24 +0800 Subject: [PATCH 573/950] Modify comments --- src/Client/ConnectionPoolWithFailover.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index c0b4965aae3..aaffe85ae2e 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -73,8 +73,8 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts ++last_used; /* Consider nested_pools.size() equals to 5 * last_used = 1 -> get_priority: 0 1 2 3 4 - * last_used = 2 -> get_priority: 5 0 1 2 3 - * last_used = 3 -> get_priority: 5 4 0 1 2 + * last_used = 2 -> get_priority: 4 0 1 2 3 + * last_used = 3 -> get_priority: 4 3 0 1 2 * ... * */ get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; }; From c82fd3491ded3999a16ae26d5c6c4ec874cd9493 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 11:06:27 +0000 Subject: [PATCH 574/950] fix fix --- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 2 +- src/Storages/FileLog/FileLogDirectoryWatcher.h | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index bd4a7543366..6d3230616f4 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -4,8 +4,8 @@ namespace DB { FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) : path(path_) - , dw(std::make_unique(*this, path, context_)) , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) + , dw(std::make_unique(*this, path, context_)) { } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index b050d9a2c39..4733dd25e0e 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -51,14 +51,16 @@ private: /// The data race is because dw create a separate thread to monitor file events /// and put into events, then if we destruct events first, the monitor thread still /// running, it may access events during events destruction, leads to data race. + /// And we should put other members before dw as well, because all of them can be + /// accessed in thread created by dw. Events events; - std::unique_ptr dw; - Poco::Logger * log; std::mutex mutex; Error error; + + std::unique_ptr dw; }; } From bb9fa383698ea8ae6a1462e401e0ec1987ee97b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 14:24:12 +0300 Subject: [PATCH 575/950] Fix header in library bridge --- src/Bridge/LibraryBridgeHelper.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index af84daf3da0..319c6c344d7 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -79,6 +80,7 @@ bool LibraryBridgeHelper::bridgeHandShake() } catch (...) { + tryLogCurrentException(log); return false; } @@ -225,7 +227,7 @@ Pipe LibraryBridgeHelper::loadKeys(const Block & requested_block) ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [requested_block, this](std::ostream & os) { WriteBufferFromOStream out_buffer(os); - auto output_format = getContext()->getOutputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, sample_block); + auto output_format = getContext()->getOutputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, out_buffer, requested_block.cloneEmpty()); formatBlock(output_format, requested_block); }; return loadBase(uri, out_stream_callback); @@ -258,7 +260,7 @@ Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTT DBMS_DEFAULT_BUFFER_SIZE, ReadWriteBufferFromHTTP::HTTPHeaderEntries{}); - auto source = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); + auto source = FormatFactory::instance().getInput(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE); source->addBuffer(std::move(read_buf_ptr)); return Pipe(std::move(source)); } From 50e2b064881e9da8d97edc12b71a0760e0135628 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 13 Oct 2021 14:25:00 +0300 Subject: [PATCH 576/950] change branch for boringssl --- .gitmodules | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..a8b94cbfa6e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -213,6 +213,7 @@ [submodule "contrib/boringssl"] path = contrib/boringssl url = https://github.com/ClickHouse-Extras/boringssl.git + branch = MergeWithUpstream [submodule "contrib/NuRaft"] path = contrib/NuRaft url = https://github.com/ClickHouse-Extras/NuRaft.git From 7cef607cb92239f1544e0afbf0da2430fb5997cf Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 13 Oct 2021 15:01:07 +0300 Subject: [PATCH 577/950] move on merge branch --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index 4c787e9d70c..486b2c0a869 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit 4c787e9d70c370d51baea714e7b73910be2a4c28 +Subproject commit 486b2c0a869fa8024c8a13e5bcefdd4a3d919947 From 562138c9fa14ec519c6ccd873d8787843eb11e6b Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Wed, 13 Oct 2021 15:01:21 +0300 Subject: [PATCH 578/950] codegen_fuzzer: removing errors on warnings from protobuf-generated code in more gentle way --- src/Parsers/fuzzers/CMakeLists.txt | 2 -- src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt | 4 +++- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/fuzzers/CMakeLists.txt b/src/Parsers/fuzzers/CMakeLists.txt index 2840dc72c0a..bb52101c847 100644 --- a/src/Parsers/fuzzers/CMakeLists.txt +++ b/src/Parsers/fuzzers/CMakeLists.txt @@ -7,7 +7,5 @@ target_link_libraries(select_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZ add_executable(create_parser_fuzzer create_parser_fuzzer.cpp ${SRCS}) target_link_libraries(create_parser_fuzzer PRIVATE clickhouse_parsers ${LIB_FUZZING_ENGINE}) -string(REPLACE " -Werror" "" CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS}") add_subdirectory(codegen_fuzzer) -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror") diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index f55bb3b3fb9..370396a5e8e 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -7,7 +7,9 @@ set(CMAKE_INCLUDE_CURRENT_DIR TRUE) add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) +set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") + target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIRS}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") -target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) \ No newline at end of file +target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) From 49ac602ee699181f85d2c285aeb08f093469ed1b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 15:06:56 +0300 Subject: [PATCH 579/950] Fix GRPC --- src/Server/GRPCServer.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 0ee9727f3d6..5d62a295dad 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -1118,7 +1119,7 @@ namespace break; if (block && !io.null_format) - output_format_processor->write(block); + output_format_processor->write(materializeBlock(block)); if (after_send_progress.elapsedMicroseconds() >= interactive_delay) { @@ -1364,7 +1365,7 @@ namespace WriteBufferFromString buf{*result.mutable_totals()}; auto format = query_context->getOutputFormat(output_format, buf, totals); format->doWritePrefix(); - format->write(totals); + format->write(materializeBlock(totals)); format->doWriteSuffix(); } @@ -1376,7 +1377,7 @@ namespace WriteBufferFromString buf{*result.mutable_extremes()}; auto format = query_context->getOutputFormat(output_format, buf, extremes); format->doWritePrefix(); - format->write(extremes); + format->write(materializeBlock(extremes)); format->doWriteSuffix(); } From d2dfbb5ab627ebe8f607fcc835e4a5a18737a303 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 15:25:44 +0300 Subject: [PATCH 580/950] Remove trash from MergeTreeReadPool --- src/Storages/MergeTree/MergeTreeReadPool.cpp | 24 ------------------- src/Storages/MergeTree/MergeTreeReadPool.h | 3 --- .../MergeTreeThreadSelectProcessor.cpp | 11 ++++----- .../00167_read_bytes_from_fs.reference | 2 ++ .../1_stateful/00167_read_bytes_from_fs.sql | 7 ++++++ 5 files changed, 13 insertions(+), 34 deletions(-) create mode 100644 tests/queries/1_stateful/00167_read_bytes_from_fs.reference create mode 100644 tests/queries/1_stateful/00167_read_bytes_from_fs.sql diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index d08cec24184..4bb247f1369 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -142,30 +142,6 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, prewhere_info && prewhere_info->remove_prewhere_column, per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); } -MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, const MarkRange & from) const -{ - MarkRanges all_part_ranges; - - /// Inefficient in presence of large number of data parts. - for (const auto & part_ranges : parts_ranges) - { - if (part_ranges.data_part.get() == &part) - { - all_part_ranges = part_ranges.ranges; - break; - } - } - if (all_part_ranges.empty()) - throw Exception("Trying to read marks range [" + std::to_string(from.begin) + ", " + std::to_string(from.end) + "] from part '" - + part.getFullPath() + "' which has no ranges in this query", ErrorCodes::LOGICAL_ERROR); - - auto begin = std::lower_bound(all_part_ranges.begin(), all_part_ranges.end(), from, [] (const auto & f, const auto & s) { return f.begin < s.begin; }); - if (begin == all_part_ranges.end()) - begin = std::prev(all_part_ranges.end()); - begin->begin = from.begin; - return MarkRanges(begin, all_part_ranges.end()); -} - Block MergeTreeReadPool::getHeader() const { return metadata_snapshot->getSampleBlockForColumns(column_names, data.getVirtuals(), data.getStorageID()); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 9949bdf86f8..380b132b806 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -85,9 +85,6 @@ public: */ void profileFeedback(const ReadBufferFromFileBase::ProfileInfo info); - /// This method tells which mark ranges we have to read if we start from @from mark range - MarkRanges getRestMarks(const IMergeTreeDataPart & part, const MarkRange & from) const; - Block getHeader() const; private: diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 4eb6bc4b2e2..6a8ef860c87 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -68,18 +68,16 @@ bool MergeTreeThreadSelectProcessor::getNewTask() if (!reader) { - auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]); - if (use_uncompressed_cache) owned_uncompressed_cache = storage.getContext()->getUncompressedCache(); owned_mark_cache = storage.getContext()->getMarkCache(); - reader = task->data_part->getReader(task->columns, metadata_snapshot, rest_mark_ranges, + reader = task->data_part->getReader(task->columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); if (prewhere_info) - pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, + pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); } @@ -88,14 +86,13 @@ bool MergeTreeThreadSelectProcessor::getNewTask() /// in other case we can reuse readers, anyway they will be "seeked" to required mark if (part_name != last_readed_part_name) { - auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]); /// retain avg_value_size_hints - reader = task->data_part->getReader(task->columns, metadata_snapshot, rest_mark_ranges, + reader = task->data_part->getReader(task->columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); if (prewhere_info) - pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, + pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, task->mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); } diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.reference b/tests/queries/1_stateful/00167_read_bytes_from_fs.reference new file mode 100644 index 00000000000..05b54da2ac7 --- /dev/null +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.reference @@ -0,0 +1,2 @@ +468426149779992039 +1 diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql new file mode 100644 index 00000000000..c3bdaea7abe --- /dev/null +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -0,0 +1,7 @@ +SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40 + +SYSTEM FLUSH LOGS; + +-- We had a bug which lead to additional compressed data read. hits compressed size if about 1.2G, but we read more then 3GB. +-- Small additional reads still possible, so we compare with about 1.5Gb. +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and type = 'QueryFinish' From 7e85b7e407838e55ee290f5747684bb6c95b44bd Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 15:27:42 +0300 Subject: [PATCH 581/950] Remove accident change --- docker/test/stateless/run.sh | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ebb72111e96..ed721690281 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,23 +45,6 @@ else sudo clickhouse start fi -echo " -set follow-fork-mode child -handle all noprint -handle SIGSEGV stop print -handle SIGBUS stop print -handle SIGABRT stop print -continue -thread apply all backtrace -detach -quit -" > script.gdb - -# FIXME Hung check may work incorrectly because of attached gdb -# 1. False positives are possible -# 2. We cannot attach another gdb to get stacktraces if some queries hung -gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From 573076588de1231f1cbc53f4718b6791027cefe8 Mon Sep 17 00:00:00 2001 From: Vitaly Orlov Date: Wed, 13 Oct 2021 16:04:08 +0300 Subject: [PATCH 582/950] Fix reference --- .../queries/0_stateless/00918_json_functions.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 4c8977114e6..8e6fc3914e0 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -101,11 +101,6 @@ hello [('a','"hello"'),('b','[-100,200,300]')] [('a','"hello"'),('b','[-100,200,300]'),('c','{"d":[121,144]}')] [('d','[121,144]')] ---JSONExtractKeys-- -['a','b'] -[] -[] -['d'] --const/non-const mixed-- a b @@ -208,6 +203,11 @@ Friday [('a','"hello"'),('b','[-100,200,300]')] [('a','"hello"'),('b','[-100,200,300]'),('c','{"d":[121,144]}')] [('d','[121,144]')] +--JSONExtractKeys-- +['a','b'] +[] +[] +['d'] --const/non-const mixed-- a b From 006a9a7c7d9e9d645c4bdf9112eafb1ed076091f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 13 Oct 2021 16:26:54 +0300 Subject: [PATCH 583/950] Print more info about memory utilization --- src/Common/ProgressIndication.cpp | 20 ++++++++++++-------- src/Common/ProgressIndication.h | 8 +++++++- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 4510952cc71..1f8fc949886 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -1,9 +1,11 @@ #include "ProgressIndication.h" +#include #include #include #include #include #include +#include "Common/formatReadable.h" #include #include #include "IO/WriteBufferFromString.h" @@ -114,16 +116,17 @@ UInt64 ProgressIndication::getApproximateCoresNumber() const }); } -UInt64 ProgressIndication::getMemoryUsage() const +ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const { - return std::accumulate(thread_data.cbegin(), thread_data.cend(), ZERO, - [](UInt64 acc, auto const & host_data) + return std::accumulate(thread_data.cbegin(), thread_data.cend(), MemoryUsage{}, + [](MemoryUsage const & acc, auto const & host_data) { - return acc + std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, + auto host_usage = std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, [](UInt64 memory, auto const & data) { return memory + data.second.memory_usage; }); + return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } @@ -202,11 +205,12 @@ void ProgressIndication::writeProgress() profiling_msg_builder << " Running " << threads_number << " threads on " << std::min(cores_number, threads_number) << " cores"; - auto memory_usage = getMemoryUsage(); + auto [memory_usage, max_host_usage] = getMemoryUsage(); if (memory_usage != 0) - profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used."; - else - profiling_msg_builder << "."; + profiling_msg_builder << " with " << formatReadableSizeWithDecimalSuffix(memory_usage) << " RAM used"; + if (thread_data.size() > 1 && max_host_usage) + profiling_msg_builder << " total (per host max: " << formatReadableSizeWithDecimalSuffix(max_host_usage) << ")"; + profiling_msg_builder << "."; profiling_msg = profiling_msg_builder.str(); } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 9b1b2b0b145..d31ed8df0ba 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -68,7 +68,13 @@ private: UInt64 getApproximateCoresNumber() const; - UInt64 getMemoryUsage() const; + struct MemoryUsage + { + UInt64 total = 0; + UInt64 max = 0; + }; + + MemoryUsage getMemoryUsage() const; /// This flag controls whether to show the progress bar. We start showing it after /// the query has been executing for 0.5 seconds, and is still less than half complete. From 168020e0db0a5217b90191b293d5212b024361b3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 14:02:32 +0000 Subject: [PATCH 584/950] fix --- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 12 ++++++------ src/Storages/FileLog/FileLogDirectoryWatcher.h | 13 +++++++------ 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 6d3230616f4..4dadbed3f09 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -30,7 +30,7 @@ const std::string & FileLogDirectoryWatcher::getPath() const return path; } -void FileLogDirectoryWatcher::onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -48,7 +48,7 @@ void FileLogDirectoryWatcher::onItemAdded(const DirectoryWatcherBase::DirectoryE } -void FileLogDirectoryWatcher::onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -71,7 +71,7 @@ void FileLogDirectoryWatcher::onItemRemoved(const DirectoryWatcherBase::Director /// So, if we record all of these events, it will use a lot of memory, and then we /// need to handle it one by one in StorageFileLog::updateFileInfos, this is unnecessary /// because it is equal to just record and handle one MODIY event -void FileLogDirectoryWatcher::onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -91,7 +91,7 @@ void FileLogDirectoryWatcher::onItemModified(const DirectoryWatcherBase::Directo } } -void FileLogDirectoryWatcher::onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -108,7 +108,7 @@ void FileLogDirectoryWatcher::onItemMovedFrom(const DirectoryWatcherBase::Direct } } -void FileLogDirectoryWatcher::onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -125,7 +125,7 @@ void FileLogDirectoryWatcher::onItemMovedTo(const DirectoryWatcherBase::Director } } -void FileLogDirectoryWatcher::onError(const Exception & e) +void FileLogDirectoryWatcher::onError(Exception e) { std::lock_guard lock(mutex); LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 4733dd25e0e..4259cc8722e 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -36,12 +36,13 @@ public: const std::string & getPath() const; - void onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev); - void onError(const Exception &); + /// Here must pass by value, otherwise will lead to stack-use-of-scope + void onItemAdded(DirectoryWatcherBase::DirectoryEvent ev); + void onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev); + void onItemModified(DirectoryWatcherBase::DirectoryEvent ev); + void onItemMovedFrom(DirectoryWatcherBase::DirectoryEvent ev); + void onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev); + void onError(Exception); private: const std::string path; From 8d7779457a325c4011ef7f88177f350fdf8e0ddd Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 13 Oct 2021 08:34:24 -0600 Subject: [PATCH 585/950] Added Checkly, updated Sipfront industery --- docs/en/introduction/adopters.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index c511bd97a7c..fe1a4dd3ea9 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -31,6 +31,7 @@ toc_title: Adopters | CardsMobile | Finance | Analytics | — | — | [VC.ru](https://vc.ru/s/cardsmobile/143449-rukovoditel-gruppy-analiza-dannyh) | | CARTO | Business Intelligence | Geo analytics | — | — | [Geospatial processing with ClickHouse](https://carto.com/blog/geospatial-processing-with-clickhouse/) | | CERN | Research | Experiment | — | — | [Press release, April 2012](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) | +| Checkly | Software Development | Analytics | — | — | [Tweet, October 2021](https://twitter.com/tim_nolet/status/1445810665743081474?s=20) | | Cisco | Networking | Traffic analysis | — | — | [Lightning talk, October 2019](https://youtu.be/-hI1vDR2oPY?t=5057) | | Citadel Securities | Finance | — | — | — | [Contribution, March 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) | | Citymobil | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) | @@ -115,7 +116,7 @@ toc_title: Adopters | seo.do | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | | SGK | Government Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | | Sina | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | -| Sipfront | Analytics | — | — | — | [Tweet, October 2021](https://twitter.com/andreasgranig/status/1446404332337913895?s=20) | +| Sipfront | Software Development | Analytics | — | — | [Tweet, October 2021](https://twitter.com/andreasgranig/status/1446404332337913895?s=20) | | SMI2 | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) | | Spark New Zealand | Telecommunications | Security Operations | — | — | [Blog Post, Feb 2020](https://blog.n0p.me/2020/02/2020-02-05-dnsmonster/) | | Splitbee | Analytics | Main Product | — | — | [Blog Post, Mai 2021](https://splitbee.io/blog/new-pricing) | From a8fa3914b530567ebb37cea9df636d38b8a18a99 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 13 Oct 2021 17:54:46 +0300 Subject: [PATCH 586/950] Fix 01939_network_send_bytes_metrics --- tests/queries/0_stateless/01939_network_send_bytes_metrics.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh b/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh index e862a273de4..840b4f54706 100755 --- a/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh +++ b/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh @@ -10,7 +10,7 @@ ${CLICKHOUSE_CLIENT} --query "SELECT number FROM numbers(1000)" > /dev/null ${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; WITH ProfileEvents['NetworkSendBytes'] AS bytes - SELECT bytes >= 8000 AND bytes < 9000 ? 1 : bytes FROM system.query_log + SELECT bytes >= 8000 AND bytes < 9500 ? 1 : bytes FROM system.query_log WHERE current_database = currentDatabase() AND query_kind = 'Select' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" ${CLICKHOUSE_CLIENT} --query "DROP TABLE t" From 886d10c3ea19c04a35ac43e175e0730a1148adc9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Oct 2021 11:58:33 +0300 Subject: [PATCH 587/950] Start server under gdb in functional tests --- docker/test/stateless/run.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ce1d1b59a55..ec0af024b8b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,6 +45,23 @@ else sudo clickhouse start fi +echo " +set follow-fork-mode child +handle all noprint +handle SIGSEGV stop print +handle SIGBUS stop print +handle SIGABRT stop print +continue +thread apply all backtrace +detach +quit +" > script.gdb + +# FIXME Hung check may work incorrectly because of attached gdb +# 1. False positives are possible +# 2. We cannot attach another gdb to get stacktraces if some queries hung +gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From 2473bc5affadd1cb397ef810a262845fd8900220 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 18:12:04 +0300 Subject: [PATCH 588/950] Fix test --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index c3bdaea7abe..341730bd82d 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -1,7 +1,7 @@ -SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40 +SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40; SYSTEM FLUSH LOGS; --- We had a bug which lead to additional compressed data read. hits compressed size if about 1.2G, but we read more then 3GB. +-- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and type = 'QueryFinish' +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and databaser=currentDatabase() and type = 'QueryFinish'; From 84eddf2498077cd2612f0fc57ca0266042030ae3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 18:12:56 +0300 Subject: [PATCH 589/950] Added tests --- .../02096_sql_user_defined_function_alias.reference | 1 + .../0_stateless/02096_sql_user_defined_function_alias.sql | 3 +++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02096_sql_user_defined_function_alias.reference create mode 100644 tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql diff --git a/tests/queries/0_stateless/02096_sql_user_defined_function_alias.reference b/tests/queries/0_stateless/02096_sql_user_defined_function_alias.reference new file mode 100644 index 00000000000..18bca00293c --- /dev/null +++ b/tests/queries/0_stateless/02096_sql_user_defined_function_alias.reference @@ -0,0 +1 @@ +a UInt16 diff --git a/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql b/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql new file mode 100644 index 00000000000..767d5945ee8 --- /dev/null +++ b/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql @@ -0,0 +1,3 @@ +CREATE FUNCTION 02096_test_function AS x -> x + 1; +DESCRIBE (SELECT 02096_test_function(1) AS a); +DROP FUNCTION 02096_test_function; \ No newline at end of file From ab5826eb402ce713b47c8b23c2569a9201f5b48f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 19:19:51 +0300 Subject: [PATCH 590/950] System data skipping indices size fix test names --- ...eference => 02028_system_data_skipping_indices_size.reference} | 0 ...dices_size.sql => 02028_system_data_skipping_indices_size.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{2028_system_data_skipping_indices_size.reference => 02028_system_data_skipping_indices_size.reference} (100%) rename tests/queries/0_stateless/{2028_system_data_skipping_indices_size.sql => 02028_system_data_skipping_indices_size.sql} (100%) diff --git a/tests/queries/0_stateless/2028_system_data_skipping_indices_size.reference b/tests/queries/0_stateless/02028_system_data_skipping_indices_size.reference similarity index 100% rename from tests/queries/0_stateless/2028_system_data_skipping_indices_size.reference rename to tests/queries/0_stateless/02028_system_data_skipping_indices_size.reference diff --git a/tests/queries/0_stateless/2028_system_data_skipping_indices_size.sql b/tests/queries/0_stateless/02028_system_data_skipping_indices_size.sql similarity index 100% rename from tests/queries/0_stateless/2028_system_data_skipping_indices_size.sql rename to tests/queries/0_stateless/02028_system_data_skipping_indices_size.sql From 98b555e7f77e0c134775fd9abb344977645f29de Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Oct 2021 19:21:11 +0300 Subject: [PATCH 591/950] Update run.sh --- docker/test/stateless/run.sh | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ec0af024b8b..ce1d1b59a55 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,23 +45,6 @@ else sudo clickhouse start fi -echo " -set follow-fork-mode child -handle all noprint -handle SIGSEGV stop print -handle SIGBUS stop print -handle SIGABRT stop print -continue -thread apply all backtrace -detach -quit -" > script.gdb - -# FIXME Hung check may work incorrectly because of attached gdb -# 1. False positives are possible -# 2. We cannot attach another gdb to get stacktraces if some queries hung -gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From 5802213722b0d489993901dfa529a84b9c5ed1b1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 19:31:59 +0300 Subject: [PATCH 592/950] PolygonDictionary support read method --- src/Dictionaries/PolygonDictionary.cpp | 72 ++++++++++++++----- src/Dictionaries/PolygonDictionary.h | 23 ++++-- .../PolygonDictionaryImplementations.cpp | 62 ++++++++-------- .../PolygonDictionaryImplementations.h | 9 +-- ...097_polygon_dictionary_store_key.reference | 1 + .../02097_polygon_dictionary_store_key.sql | 38 ++++++++++ 6 files changed, 142 insertions(+), 63 deletions(-) create mode 100644 tests/queries/0_stateless/02097_polygon_dictionary_store_key.reference create mode 100644 tests/queries/0_stateless/02097_polygon_dictionary_store_key.sql diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 06a705a351e..244ee9da03b 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -7,7 +7,8 @@ #include #include #include -#include +#include +#include #include #include @@ -28,14 +29,12 @@ IPolygonDictionary::IPolygonDictionary( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_) + Configuration configuration_) : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr(std::move(source_ptr_)) , dict_lifetime(dict_lifetime_) - , input_type(input_type_) - , point_type(point_type_) + , configuration(configuration_) { setup(); loadData(); @@ -55,7 +54,7 @@ ColumnPtr IPolygonDictionary::getColumn( DefaultValueProvider default_value_provider(attribute.null_value, default_values_column); size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; - const auto & attribute_values_column = attributes[attribute_index]; + const auto & attribute_values_column = attributes_columns[attribute_index]; auto result = attribute_values_column->cloneEmpty(); result->reserve(requested_key_points.size()); @@ -119,21 +118,52 @@ ColumnPtr IPolygonDictionary::getColumn( return result; } -Pipe IPolygonDictionary::read(const Names &, size_t) const +Pipe IPolygonDictionary::read(const Names & column_names, size_t) const { - // TODO: In order for this to work one would first have to support retrieving arrays from dictionaries. - // I believe this is a separate task done by some other people. - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Reading the dictionary is not allowed"); + if (!configuration.store_polygon_key_column) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Set `store_polygon_key_column` setting in dictionary configuration to true to support reading from PolygonDictionary."); + + const auto & dictionary_structure_keys = *dict_struct.key; + const auto & dictionary_key_attribute = dictionary_structure_keys[0]; + + ColumnsWithTypeAndName result_columns; + result_columns.reserve(column_names.size()); + + for (const auto & column_name : column_names) { + ColumnWithTypeAndName column_with_type; + + if (column_name == dictionary_key_attribute.name) { + column_with_type.column = key_attribute_column; + column_with_type.type = dictionary_key_attribute.type; + } else { + const auto & dictionary_attribute = dict_struct.getAttribute(column_name); + size_t attribute_index = dict_struct.attribute_name_to_index.find(dictionary_attribute.name)->second; + + column_with_type.column = attributes_columns[attribute_index]; + column_with_type.type = dictionary_attribute.type; + } + + column_with_type.name = column_name; + + result_columns.emplace_back(column_with_type); + } + + auto source = std::make_shared(Block(result_columns)); + return Pipe(std::move(source)); } void IPolygonDictionary::setup() { - attributes.reserve(dict_struct.attributes.size()); + const auto & dictionary_structure_keys = *dict_struct.key; + key_attribute_column = dictionary_structure_keys[0].type->createColumn(); + + attributes_columns.reserve(dict_struct.attributes.size()); for (const auto & attribute : dict_struct.attributes) { auto column = attribute.type->createColumn(); - attributes.emplace_back(std::move(column)); + attributes_columns.emplace_back(std::move(column)); if (attribute.hierarchical) throw Exception(ErrorCodes::TYPE_MISMATCH, @@ -147,20 +177,24 @@ void IPolygonDictionary::blockToAttributes(const DB::Block & block) const auto rows = block.rows(); size_t skip_key_column_offset = 1; - for (size_t i = 0; i < attributes.size(); ++i) + for (size_t i = 0; i < attributes_columns.size(); ++i) { const auto & block_column = block.safeGetByPosition(i + skip_key_column_offset); const auto & column = block_column.column; - attributes[i]->assumeMutable()->insertRangeFrom(*column, 0, column->size()); + attributes_columns[i]->assumeMutable()->insertRangeFrom(*column, 0, column->size()); } /** Multi-polygons could cause bigger sizes, but this is better than nothing. */ polygons.reserve(polygons.size() + rows); polygon_index_to_attribute_value_index.reserve(polygon_index_to_attribute_value_index.size() + rows); - const auto & key = block.safeGetByPosition(0).column; - extractPolygons(key); + const auto & key_column = block.safeGetByPosition(0).column; + + if (configuration.store_polygon_key_column) + key_attribute_column->assumeMutable()->insertRangeFrom(*key_column, 0, key_column->size()); + + extractPolygons(key_column); } void IPolygonDictionary::loadData() @@ -210,7 +244,7 @@ void IPolygonDictionary::calculateBytesAllocated() { /// Index allocated by subclass not counted because it take a small part in relation to attributes and polygons - for (const auto & column : attributes) + for (const auto & column : attributes_columns) bytes_allocated += column->allocatedBytes(); for (auto & polygon : polygons) @@ -487,7 +521,7 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) Offset offset; const IColumn * points_collection = nullptr; - switch (input_type) + switch (configuration.input_type) { case InputType::MultiPolygon: points_collection = unrollMultiPolygons(column, offset); @@ -504,7 +538,7 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) /** Adding the first empty polygon */ data.addPolygon(true); - switch (point_type) + switch (configuration.point_type) { case PointType::Array: handlePointsReprByArrays(points_collection, data, offset); diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 9ab82890c49..73b782f7692 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -50,13 +50,22 @@ public: Tuple, }; + struct Configuration + { + /// Store polygon key column. That will allow to read columns from polygon dictionary. + bool store_polygon_key_column = false; + + InputType input_type = InputType::MultiPolygon; + + PointType point_type = PointType::Array; + }; + IPolygonDictionary( const StorageID & dict_id_, const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_); + Configuration configuration_); std::string getTypeName() const override { return "Polygon"; } @@ -74,7 +83,7 @@ public: double getHitRate() const override { return 1.0; } - size_t getElementCount() const override { return attributes.empty() ? 0 : attributes.front()->size(); } + size_t getElementCount() const override { return attributes_columns.empty() ? 0 : attributes_columns.front()->size(); } double getLoadFactor() const override { return 1.0; } @@ -120,9 +129,7 @@ protected: const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; - - const InputType input_type; - const PointType point_type; + const Configuration configuration; private: /** Helper functions for loading the data from the configuration. @@ -146,7 +153,9 @@ private: ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; - Columns attributes; + ColumnPtr key_attribute_column; + + Columns attributes_columns; size_t bytes_allocated = 0; mutable std::atomic query_count{0}; diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index bd09e9df073..93ba67a59e9 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -22,9 +22,8 @@ PolygonDictionarySimple::PolygonDictionarySimple( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_): - IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_) + Configuration configuration_) + : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, configuration_) { } @@ -35,8 +34,7 @@ std::shared_ptr PolygonDictionarySimple::clone() const this->dict_struct, this->source_ptr->clone(), this->dict_lifetime, - this->input_type, - this->point_type); + this->configuration); } bool PolygonDictionarySimple::find(const Point & point, size_t & polygon_index) const @@ -59,11 +57,10 @@ PolygonDictionaryIndexEach::PolygonDictionaryIndexEach( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_, + Configuration configuration_, int min_intersections_, int max_depth_) - : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_), + : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, configuration_), grid(min_intersections_, max_depth_, polygons), min_intersections(min_intersections_), max_depth(max_depth_) @@ -84,8 +81,7 @@ std::shared_ptr PolygonDictionaryIndexEach::clone() con this->dict_struct, this->source_ptr->clone(), this->dict_lifetime, - this->input_type, - this->point_type, + this->configuration, this->min_intersections, this->max_depth); } @@ -118,11 +114,10 @@ PolygonDictionaryIndexCell::PolygonDictionaryIndexCell( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_, + Configuration configuration_, size_t min_intersections_, size_t max_depth_) - : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, input_type_, point_type_), + : IPolygonDictionary(dict_id_, dict_struct_, std::move(source_ptr_), dict_lifetime_, configuration_), index(min_intersections_, max_depth_, polygons), min_intersections(min_intersections_), max_depth(max_depth_) @@ -136,8 +131,7 @@ std::shared_ptr PolygonDictionaryIndexCell::clone() con this->dict_struct, this->source_ptr->clone(), this->dict_lifetime, - this->input_type, - this->point_type, + this->configuration, this->min_intersections, this->max_depth); } @@ -179,35 +173,37 @@ DictionaryPtr createLayout(const std::string & , throw Exception(ErrorCodes::BAD_ARGUMENTS, "The 'key' should consist of a single attribute for a polygon dictionary"); - IPolygonDictionary::InputType input_type; - IPolygonDictionary::PointType point_type; + IPolygonDictionary::Configuration configuration; + const auto key_type = (*dict_struct.key)[0].type; const auto f64 = std::make_shared(); const auto multi_polygon_array = DataTypeArray(std::make_shared(std::make_shared(std::make_shared(f64)))); const auto multi_polygon_tuple = DataTypeArray(std::make_shared(std::make_shared(std::make_shared(std::vector{f64, f64})))); const auto simple_polygon_array = DataTypeArray(std::make_shared(f64)); const auto simple_polygon_tuple = DataTypeArray(std::make_shared(std::vector{f64, f64})); + if (key_type->equals(multi_polygon_array)) { - input_type = IPolygonDictionary::InputType::MultiPolygon; - point_type = IPolygonDictionary::PointType::Array; + configuration.input_type = IPolygonDictionary::InputType::MultiPolygon; + configuration.point_type = IPolygonDictionary::PointType::Array; } else if (key_type->equals(multi_polygon_tuple)) { - input_type = IPolygonDictionary::InputType::MultiPolygon; - point_type = IPolygonDictionary::PointType::Tuple; + configuration.input_type = IPolygonDictionary::InputType::MultiPolygon; + configuration.point_type = IPolygonDictionary::PointType::Tuple; } else if (key_type->equals(simple_polygon_array)) { - input_type = IPolygonDictionary::InputType::SimplePolygon; - point_type = IPolygonDictionary::PointType::Array; + configuration.input_type = IPolygonDictionary::InputType::SimplePolygon; + configuration.point_type = IPolygonDictionary::PointType::Array; } else if (key_type->equals(simple_polygon_tuple)) { - input_type = IPolygonDictionary::InputType::SimplePolygon; - point_type = IPolygonDictionary::PointType::Tuple; + configuration.input_type = IPolygonDictionary::InputType::SimplePolygon; + configuration.point_type = IPolygonDictionary::PointType::Tuple; } else + { throw Exception(ErrorCodes::BAD_ARGUMENTS, "The key type {} is not one of the following allowed types for a polygon dictionary: {} {} {} {} ", key_type->getName(), @@ -215,6 +211,14 @@ DictionaryPtr createLayout(const std::string & , multi_polygon_tuple.getName(), simple_polygon_array.getName(), simple_polygon_tuple.getName()); + } + + const auto & layout_prefix = config_prefix + ".layout"; + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(layout_prefix, keys); + const auto & dict_prefix = layout_prefix + "." + keys.front(); + + configuration.store_polygon_key_column = config.getUInt(dict_prefix + ".store_polygon_key_column", false); if (dict_struct.range_min || dict_struct.range_max) throw Exception(ErrorCodes::BAD_ARGUMENTS, @@ -228,16 +232,12 @@ DictionaryPtr createLayout(const std::string & , if constexpr (std::is_same_v || std::is_same_v) { - const auto & layout_prefix = config_prefix + ".layout"; - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(layout_prefix, keys); - const auto & dict_prefix = layout_prefix + "." + keys.front(); size_t max_depth = config.getUInt(dict_prefix + ".max_depth", PolygonDictionary::kMaxDepthDefault); size_t min_intersections = config.getUInt(dict_prefix + ".min_intersections", PolygonDictionary::kMinIntersectionsDefault); - return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type, min_intersections, max_depth); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, configuration, min_intersections, max_depth); } else - return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, input_type, point_type); + return std::make_unique(dict_id, dict_struct, std::move(source_ptr), dict_lifetime, configuration); } void registerDictionaryPolygon(DictionaryFactory & factory) diff --git a/src/Dictionaries/PolygonDictionaryImplementations.h b/src/Dictionaries/PolygonDictionaryImplementations.h index b49bf3f4d6c..912d501bcde 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.h +++ b/src/Dictionaries/PolygonDictionaryImplementations.h @@ -21,8 +21,7 @@ public: const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_); + Configuration configuration_); std::shared_ptr clone() const override; @@ -44,8 +43,7 @@ public: const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_, + Configuration configuration_, int min_intersections_, int max_depth_); @@ -73,8 +71,7 @@ public: const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, DictionaryLifetime dict_lifetime_, - InputType input_type_, - PointType point_type_, + Configuration configuration_, size_t min_intersections_, size_t max_depth_); diff --git a/tests/queries/0_stateless/02097_polygon_dictionary_store_key.reference b/tests/queries/0_stateless/02097_polygon_dictionary_store_key.reference new file mode 100644 index 00000000000..f44a7c2a068 --- /dev/null +++ b/tests/queries/0_stateless/02097_polygon_dictionary_store_key.reference @@ -0,0 +1 @@ +[[[(3,1),(0,1),(0,-1),(3,-1)]]] Value diff --git a/tests/queries/0_stateless/02097_polygon_dictionary_store_key.sql b/tests/queries/0_stateless/02097_polygon_dictionary_store_key.sql new file mode 100644 index 00000000000..95557da481e --- /dev/null +++ b/tests/queries/0_stateless/02097_polygon_dictionary_store_key.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS polygons_test_table; +CREATE TABLE polygons_test_table +( + key Array(Array(Array(Tuple(Float64, Float64)))), + name String +) ENGINE = TinyLog; + +INSERT INTO polygons_test_table VALUES ([[[(3, 1), (0, 1), (0, -1), (3, -1)]]], 'Value'); + +DROP DICTIONARY IF EXISTS polygons_test_dictionary_no_option; +CREATE DICTIONARY polygons_test_dictionary_no_option +( + key Array(Array(Array(Tuple(Float64, Float64)))), + name String +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE 'polygons_test_table')) +LAYOUT(POLYGON()) +LIFETIME(0); + +SELECT * FROM polygons_test_dictionary_no_option; -- {serverError 1} + +DROP DICTIONARY IF EXISTS polygons_test_dictionary; +CREATE DICTIONARY polygons_test_dictionary +( + key Array(Array(Array(Tuple(Float64, Float64)))), + name String +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(TABLE 'polygons_test_table')) +LAYOUT(POLYGON(STORE_POLYGON_KEY_COLUMN 1)) +LIFETIME(0); + +SELECT * FROM polygons_test_dictionary; + +DROP DICTIONARY polygons_test_dictionary_no_option; +DROP DICTIONARY polygons_test_dictionary; +DROP TABLE polygons_test_table; From d1a184c50025f22be44ac50e40a9c0491b605211 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 19:37:16 +0300 Subject: [PATCH 593/950] Fixed style check --- src/Dictionaries/PolygonDictionary.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 244ee9da03b..7a34a9c2b25 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -130,13 +130,17 @@ Pipe IPolygonDictionary::read(const Names & column_names, size_t) const ColumnsWithTypeAndName result_columns; result_columns.reserve(column_names.size()); - for (const auto & column_name : column_names) { + for (const auto & column_name : column_names) + { ColumnWithTypeAndName column_with_type; - if (column_name == dictionary_key_attribute.name) { + if (column_name == dictionary_key_attribute.name) + { column_with_type.column = key_attribute_column; column_with_type.type = dictionary_key_attribute.type; - } else { + } + else + { const auto & dictionary_attribute = dict_struct.getAttribute(column_name); size_t attribute_index = dict_struct.attribute_name_to_index.find(dictionary_attribute.name)->second; From 8851cb8459fcb490acec6ce7af33a709b0b11539 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 13 Oct 2021 11:37:56 +0800 Subject: [PATCH 594/950] Use upstream replxx --- .gitmodules | 2 +- contrib/replxx | 2 +- src/Client/ClientBaseHelpers.cpp | 44 ++++++++++++++++---------------- 3 files changed, 24 insertions(+), 24 deletions(-) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..f9758a69956 100644 --- a/.gitmodules +++ b/.gitmodules @@ -140,7 +140,7 @@ url = https://github.com/ClickHouse-Extras/libc-headers.git [submodule "contrib/replxx"] path = contrib/replxx - url = https://github.com/ClickHouse-Extras/replxx.git + url = https://github.com/AmokHuginnsson/replxx.git [submodule "contrib/avro"] path = contrib/avro url = https://github.com/ClickHouse-Extras/avro.git diff --git a/contrib/replxx b/contrib/replxx index f97765df14f..89abeea7516 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit f97765df14f4a6236d69b8f14b53ef2051ebd95a +Subproject commit 89abeea7516a2a9b6aad7bfecc132f608ff14a3d diff --git a/src/Client/ClientBaseHelpers.cpp b/src/Client/ClientBaseHelpers.cpp index a530e48ee35..e1c1481c5b4 100644 --- a/src/Client/ClientBaseHelpers.cpp +++ b/src/Client/ClientBaseHelpers.cpp @@ -109,29 +109,29 @@ void highlight(const String & query, std::vector & colors {TokenType::OpeningSquareBracket, Replxx::Color::BROWN}, {TokenType::ClosingSquareBracket, Replxx::Color::BROWN}, {TokenType::DoubleColon, Replxx::Color::BROWN}, - {TokenType::OpeningCurlyBrace, Replxx::Color::INTENSE}, - {TokenType::ClosingCurlyBrace, Replxx::Color::INTENSE}, + {TokenType::OpeningCurlyBrace, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::ClosingCurlyBrace, replxx::color::bold(Replxx::Color::DEFAULT)}, - {TokenType::Comma, Replxx::Color::INTENSE}, - {TokenType::Semicolon, Replxx::Color::INTENSE}, - {TokenType::Dot, Replxx::Color::INTENSE}, - {TokenType::Asterisk, Replxx::Color::INTENSE}, + {TokenType::Comma, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Semicolon, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Dot, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Asterisk, replxx::color::bold(Replxx::Color::DEFAULT)}, {TokenType::HereDoc, Replxx::Color::CYAN}, - {TokenType::Plus, Replxx::Color::INTENSE}, - {TokenType::Minus, Replxx::Color::INTENSE}, - {TokenType::Slash, Replxx::Color::INTENSE}, - {TokenType::Percent, Replxx::Color::INTENSE}, - {TokenType::Arrow, Replxx::Color::INTENSE}, - {TokenType::QuestionMark, Replxx::Color::INTENSE}, - {TokenType::Colon, Replxx::Color::INTENSE}, - {TokenType::Equals, Replxx::Color::INTENSE}, - {TokenType::NotEquals, Replxx::Color::INTENSE}, - {TokenType::Less, Replxx::Color::INTENSE}, - {TokenType::Greater, Replxx::Color::INTENSE}, - {TokenType::LessOrEquals, Replxx::Color::INTENSE}, - {TokenType::GreaterOrEquals, Replxx::Color::INTENSE}, - {TokenType::Concatenation, Replxx::Color::INTENSE}, - {TokenType::At, Replxx::Color::INTENSE}, + {TokenType::Plus, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Minus, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Slash, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Percent, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Arrow, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::QuestionMark, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Colon, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Equals, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::NotEquals, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Less, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Greater, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::LessOrEquals, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::GreaterOrEquals, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::Concatenation, replxx::color::bold(Replxx::Color::DEFAULT)}, + {TokenType::At, replxx::color::bold(Replxx::Color::DEFAULT)}, {TokenType::DoubleAt, Replxx::Color::MAGENTA}, {TokenType::EndOfStream, Replxx::Color::DEFAULT}, @@ -142,7 +142,7 @@ void highlight(const String & query, std::vector & colors {TokenType::ErrorDoubleQuoteIsNotClosed, Replxx::Color::RED}, {TokenType::ErrorSinglePipeMark, Replxx::Color::RED}, {TokenType::ErrorWrongNumber, Replxx::Color::RED}, - { TokenType::ErrorMaxQuerySizeExceeded, Replxx::Color::RED }}; + {TokenType::ErrorMaxQuerySizeExceeded, Replxx::Color::RED}}; const Replxx::Color unknown_token_color = Replxx::Color::RED; From b24b458df7a773925324df83bc646ba83d07be8d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 20:13:30 +0300 Subject: [PATCH 595/950] Update HTTPDictionarySource.cpp --- src/Dictionaries/HTTPDictionarySource.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 6cb4d52744a..8c76f0ff0dd 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -134,7 +134,7 @@ Pipe HTTPDictionarySource::loadIds(const std::vector & ids) ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); - auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, sample_block); + auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, block.cloneEmpty()); formatBlock(output_format, block); }; @@ -161,7 +161,7 @@ Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vect ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); - auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, sample_block); + auto output_format = context->getOutputFormatParallelIfPossible(configuration.format, out_buffer, block.cloneEmpty()); formatBlock(output_format, block); }; From 91084895cee130a36ddb6e15c9727e6da6b607bf Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Wed, 13 Oct 2021 20:18:51 +0300 Subject: [PATCH 596/950] attemp to fix build --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index 486b2c0a869..a139bb3cb95 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit 486b2c0a869fa8024c8a13e5bcefdd4a3d919947 +Subproject commit a139bb3cb9598c7d92dc69aa6962e3ea4fd18982 From 3d0e595579a641f30b46b3577a66059d87a15e0e Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 22:49:21 +0530 Subject: [PATCH 597/950] revise compile flag --- cmake/cpu_features.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 41a3dc26d5a..330ab10f1bf 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -163,8 +163,8 @@ else () endif () #Limit avx2/avx512 flag for specific source build + set (X86_INTRINSICS_FLAGS "") if (ENABLE_AVX2_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") if (HAVE_BMI) set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") endif () From 2957971ee30b9507cb73d6b6c90a1ebee87914fd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Oct 2021 21:22:02 +0300 Subject: [PATCH 598/950] Remove some last streams. --- src/Core/ExternalTable.cpp | 1 - src/DataStreams/BlockIO.cpp | 1 - .../MaterializingBlockOutputStream.h | 34 --- src/DataStreams/OneBlockInputStream.h | 41 ---- src/DataStreams/SquashingBlockInputStream.cpp | 32 --- src/DataStreams/SquashingBlockInputStream.h | 31 --- .../SquashingBlockOutputStream.cpp | 54 ----- src/DataStreams/SquashingBlockOutputStream.h | 39 ---- .../gtest_blocks_size_merging_streams.cpp | 32 +-- src/Databases/DatabaseReplicated.h | 1 - .../MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Dictionaries/LibraryDictionarySource.cpp | 1 - src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Formats/FormatFactory.cpp | 1 - src/Interpreters/Context.cpp | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 2 - src/Interpreters/InterpreterSelectQuery.cpp | 51 ++--- src/Interpreters/InterpreterSelectQuery.h | 4 +- src/Interpreters/MergeJoin.cpp | 7 +- src/Interpreters/MutationsInterpreter.cpp | 1 - .../PipelineExecutingBlockInputStream.cpp | 124 ----------- .../PipelineExecutingBlockInputStream.h | 44 ---- .../Formats/InputStreamFromInputFormat.h | 67 ------ src/Processors/Pipe.cpp | 10 +- src/Processors/QueryPipeline.cpp | 10 +- src/Processors/QueryPipelineBuilder.cpp | 10 +- src/Processors/QueryPlan/ExpressionStep.cpp | 1 - .../Sources/SourceFromInputStream.cpp | 195 ------------------ .../Sources/SourceFromInputStream.h | 77 ------- .../Transforms/AggregatingTransform.h | 3 - src/Storages/HDFS/StorageHDFS.cpp | 16 +- src/Storages/Kafka/KafkaSource.cpp | 2 - src/Storages/Kafka/StorageKafka.cpp | 1 - src/Storages/MergeTree/MergeTask.cpp | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 9 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 - .../MergeTree/MergeTreeDataWriter.cpp | 2 - src/Storages/MergeTree/MutateTask.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 1 - .../MaterializedPostgreSQLConsumer.h | 1 - .../StorageMaterializedPostgreSQL.cpp | 1 - src/Storages/RabbitMQ/RabbitMQSource.cpp | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 - .../RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDictionary.cpp | 1 - src/Storages/StorageDistributed.cpp | 1 - src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageInput.cpp | 1 - src/Storages/StorageMaterializedView.cpp | 1 - src/Storages/StorageMerge.cpp | 4 +- src/Storages/StorageMongoDB.cpp | 1 - src/Storages/StorageMySQL.cpp | 1 - src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 2 - src/Storages/StorageSQLite.cpp | 1 - src/Storages/StorageURL.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 1 - 58 files changed, 67 insertions(+), 874 deletions(-) delete mode 100644 src/DataStreams/MaterializingBlockOutputStream.h delete mode 100644 src/DataStreams/OneBlockInputStream.h delete mode 100644 src/DataStreams/SquashingBlockInputStream.cpp delete mode 100644 src/DataStreams/SquashingBlockInputStream.h delete mode 100644 src/DataStreams/SquashingBlockOutputStream.cpp delete mode 100644 src/DataStreams/SquashingBlockOutputStream.h delete mode 100644 src/Processors/Executors/PipelineExecutingBlockInputStream.cpp delete mode 100644 src/Processors/Executors/PipelineExecutingBlockInputStream.h delete mode 100644 src/Processors/Formats/InputStreamFromInputFormat.h delete mode 100644 src/Processors/Sources/SourceFromInputStream.cpp delete mode 100644 src/Processors/Sources/SourceFromInputStream.h diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 55ad748868e..7619d60d84e 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -12,7 +12,6 @@ #include #include -#include #include #include #include diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 5f1abdaf806..692b69388ea 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -1,6 +1,5 @@ #include #include -#include namespace DB { diff --git a/src/DataStreams/MaterializingBlockOutputStream.h b/src/DataStreams/MaterializingBlockOutputStream.h deleted file mode 100644 index 64c2bc12a57..00000000000 --- a/src/DataStreams/MaterializingBlockOutputStream.h +++ /dev/null @@ -1,34 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Converts columns-constants to full columns ("materializes" them). - */ -class MaterializingBlockOutputStream : public IBlockOutputStream -{ -public: - MaterializingBlockOutputStream(const BlockOutputStreamPtr & output_, const Block & header_) - : output{output_}, header(header_) {} - - Block getHeader() const override { return header; } - void write(const Block & block) override { output->write(materializeBlock(block)); } - void flush() override { output->flush(); } - void writePrefix() override { output->writePrefix(); } - void writeSuffix() override { output->writeSuffix(); } - void setRowsBeforeLimit(size_t rows_before_limit) override { output->setRowsBeforeLimit(rows_before_limit); } - void setTotals(const Block & totals) override { output->setTotals(materializeBlock(totals)); } - void setExtremes(const Block & extremes) override { output->setExtremes(materializeBlock(extremes)); } - void onProgress(const Progress & progress) override { output->onProgress(progress); } - String getContentType() const override { return output->getContentType(); } - -private: - BlockOutputStreamPtr output; - Block header; -}; - -} diff --git a/src/DataStreams/OneBlockInputStream.h b/src/DataStreams/OneBlockInputStream.h deleted file mode 100644 index d401082ce62..00000000000 --- a/src/DataStreams/OneBlockInputStream.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/** A stream of blocks from which you can read one block. - */ -class OneBlockInputStream : public IBlockInputStream -{ -public: - explicit OneBlockInputStream(Block block_) : block(std::move(block_)) { block.checkNumberOfRows(); } - - String getName() const override { return "One"; } - - Block getHeader() const override - { - Block res; - for (const auto & elem : block) - res.insert({ elem.column->cloneEmpty(), elem.type, elem.name }); - return res; - } - -protected: - Block readImpl() override - { - if (has_been_read) - return Block(); - - has_been_read = true; - return block; - } - -private: - Block block; - bool has_been_read = false; -}; - -} diff --git a/src/DataStreams/SquashingBlockInputStream.cpp b/src/DataStreams/SquashingBlockInputStream.cpp deleted file mode 100644 index e13dee37008..00000000000 --- a/src/DataStreams/SquashingBlockInputStream.cpp +++ /dev/null @@ -1,32 +0,0 @@ -#include - - -namespace DB -{ - -SquashingBlockInputStream::SquashingBlockInputStream( - const BlockInputStreamPtr & src, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory) - : header(src->getHeader()), transform(min_block_size_rows, min_block_size_bytes, reserve_memory) -{ - children.emplace_back(src); -} - - -Block SquashingBlockInputStream::readImpl() -{ - while (!all_read) - { - Block block = children[0]->read(); - if (!block) - all_read = true; - - auto squashed_block = transform.add(std::move(block)); - if (squashed_block) - { - return squashed_block; - } - } - return {}; -} - -} diff --git a/src/DataStreams/SquashingBlockInputStream.h b/src/DataStreams/SquashingBlockInputStream.h deleted file mode 100644 index c2732d520cc..00000000000 --- a/src/DataStreams/SquashingBlockInputStream.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Merging consecutive blocks of stream to specified minimum size. - */ -class SquashingBlockInputStream : public IBlockInputStream -{ -public: - SquashingBlockInputStream(const BlockInputStreamPtr & src, size_t min_block_size_rows, size_t min_block_size_bytes, - bool reserve_memory = false); - - String getName() const override { return "Squashing"; } - - Block getHeader() const override { return header; } - -protected: - Block readImpl() override; - -private: - Block header; - SquashingTransform transform; - bool all_read = false; -}; - -} diff --git a/src/DataStreams/SquashingBlockOutputStream.cpp b/src/DataStreams/SquashingBlockOutputStream.cpp deleted file mode 100644 index ab12f66590f..00000000000 --- a/src/DataStreams/SquashingBlockOutputStream.cpp +++ /dev/null @@ -1,54 +0,0 @@ -#include - - -namespace DB -{ - -SquashingBlockOutputStream::SquashingBlockOutputStream(BlockOutputStreamPtr dst, Block header_, size_t min_block_size_rows, size_t min_block_size_bytes) - : output(std::move(dst)), header(std::move(header_)), transform(min_block_size_rows, min_block_size_bytes) -{ -} - - -void SquashingBlockOutputStream::write(const Block & block) -{ - auto squashed_block = transform.add(block); - if (squashed_block) - output->write(squashed_block); -} - - -void SquashingBlockOutputStream::finalize() -{ - if (all_written) - return; - - all_written = true; - - auto squashed_block = transform.add({}); - if (squashed_block) - output->write(squashed_block); -} - - -void SquashingBlockOutputStream::flush() -{ - if (!disable_flush) - finalize(); - output->flush(); -} - - -void SquashingBlockOutputStream::writePrefix() -{ - output->writePrefix(); -} - - -void SquashingBlockOutputStream::writeSuffix() -{ - finalize(); - output->writeSuffix(); -} - -} diff --git a/src/DataStreams/SquashingBlockOutputStream.h b/src/DataStreams/SquashingBlockOutputStream.h deleted file mode 100644 index 7828ad7e96d..00000000000 --- a/src/DataStreams/SquashingBlockOutputStream.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Merging consecutive blocks of stream to specified minimum size. - */ -class SquashingBlockOutputStream : public IBlockOutputStream -{ -public: - SquashingBlockOutputStream(BlockOutputStreamPtr dst, Block header_, size_t min_block_size_rows, size_t min_block_size_bytes); - - Block getHeader() const override { return header; } - void write(const Block & block) override; - - void flush() override; - void writePrefix() override; - void writeSuffix() override; - - /// Don't write blocks less than specified size even when flush method was called by user. - void disableFlush() { disable_flush = true; } - -private: - BlockOutputStreamPtr output; - Block header; - - SquashingTransform transform; - bool all_written = false; - - void finalize(); - - bool disable_flush = false; -}; - -} diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp index 71cf41fcbab..7625fe8fa09 100644 --- a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include using namespace DB; @@ -88,15 +88,18 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) pipe.addTransform(std::move(transform)); QueryPipeline pipeline(std::move(pipe)); - pipeline.setNumThreads(1); - auto stream = std::make_shared(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); size_t total_rows = 0; - auto block1 = stream->read(); - auto block2 = stream->read(); - auto block3 = stream->read(); + Block block1; + Block block2; + Block block3; + executor.pull(block1); + executor.pull(block2); + executor.pull(block3); - EXPECT_EQ(stream->read(), Block()); + Block tmp_block; + ASSERT_FALSE(executor.pull(tmp_block)); for (const auto & block : {block1, block2, block3}) total_rows += block.rows(); @@ -132,14 +135,17 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) pipe.addTransform(std::move(transform)); QueryPipeline pipeline(std::move(pipe)); - pipeline.setNumThreads(1); - auto stream = std::make_shared(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); - auto block1 = stream->read(); - auto block2 = stream->read(); - auto block3 = stream->read(); + Block block1; + Block block2; + Block block3; + executor.pull(block1); + executor.pull(block2); + executor.pull(block3); - EXPECT_EQ(stream->read(), Block()); + Block tmp_block; + ASSERT_FALSE(executor.pull(tmp_block)); EXPECT_EQ(block1.rows(), (1000 + 1500 + 1400) / 3); EXPECT_EQ(block2.rows(), (1000 + 1500 + 1400) / 3); diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 60526a1e5b0..aecbc1474f8 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -5,7 +5,6 @@ #include #include #include -#include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 5a714645978..f4a5b6b5e4e 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -14,7 +14,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 551bb1ee2dd..f117cfb179e 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -1,6 +1,5 @@ #include "LibraryDictionarySource.h" -#include #include #include #include diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 80081e67b42..b0be90c26a5 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -18,6 +17,7 @@ #include "registerDictionaries.h" #include #include +#include namespace DB diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d3ff5cbf8a7..ed9f9d52b94 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6e729eb2370..46fa31e2738 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index a36941ea07a..8a31917caef 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -20,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85cc889319f..b4ffa15a869 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,4 +1,3 @@ -#include #include #include @@ -64,7 +63,6 @@ #include #include #include -#include #include #include #include @@ -158,24 +156,16 @@ InterpreterSelectQuery::InterpreterSelectQuery( ContextPtr context_, const SelectQueryOptions & options_, const Names & required_result_column_names_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, nullptr, options_, required_result_column_names_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_) { } -InterpreterSelectQuery::InterpreterSelectQuery( - const ASTPtr & query_ptr_, - ContextPtr context_, - const BlockInputStreamPtr & input_, - const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, input_, std::nullopt, nullptr, options_.copy().noSubquery()) -{} - InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, Pipe input_pipe_, const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) + : InterpreterSelectQuery(query_ptr_, context_, std::move(input_pipe_), nullptr, options_.copy().noSubquery()) {} InterpreterSelectQuery::InterpreterSelectQuery( @@ -184,7 +174,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( const StoragePtr & storage_, const StorageMetadataPtr & metadata_snapshot_, const SelectQueryOptions & options_) - : InterpreterSelectQuery(query_ptr_, context_, nullptr, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, storage_, options_.copy().noSubquery(), {}, metadata_snapshot_) {} InterpreterSelectQuery::~InterpreterSelectQuery() = default; @@ -268,7 +258,6 @@ static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, - const BlockInputStreamPtr & input_, std::optional input_pipe_, const StoragePtr & storage_, const SelectQueryOptions & options_, @@ -277,7 +266,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// NOTE: the query almost always should be cloned because it will be modified during analysis. : IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_) , storage(storage_) - , input(input_) , input_pipe(std::move(input_pipe_)) , log(&Poco::Logger::get("InterpreterSelectQuery")) , metadata_snapshot(metadata_snapshot_) @@ -294,13 +282,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(), ErrorCodes::TOO_DEEP_SUBQUERIES); - bool has_input = input || input_pipe; - if (input) - { - /// Read from prepared input. - source_header = input->getHeader(); - } - else if (input_pipe) + bool has_input = input_pipe != std::nullopt; + if (input_pipe) { /// Read from prepared input. source_header = input_pipe->getHeader(); @@ -450,17 +433,17 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!options.only_analyze) { - if (query.sampleSize() && (input || input_pipe || !storage || !storage->supportsSampling())) + if (query.sampleSize() && (input_pipe || !storage || !storage->supportsSampling())) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); - if (query.final() && (input || input_pipe || !storage || !storage->supportsFinal())) + if (query.final() && (input_pipe || !storage || !storage->supportsFinal())) throw Exception( - (!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", + (!input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support FINAL" : "Illegal FINAL", ErrorCodes::ILLEGAL_FINAL); - if (query.prewhere() && (input || input_pipe || !storage || !storage->supportsPrewhere())) + if (query.prewhere() && (input_pipe || !storage || !storage->supportsPrewhere())) throw Exception( - (!input && !input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", + (!input_pipe && storage) ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE); /// Save the new temporary tables in the query context @@ -578,7 +561,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { - executeImpl(query_plan, input, std::move(input_pipe)); + executeImpl(query_plan, std::move(input_pipe)); /// We must guarantee that result structure is the same as in getSampleBlock() /// @@ -926,7 +909,7 @@ static bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query) } -void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe) +void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional prepared_pipe) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -1010,13 +993,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu } else { - if (prepared_input) - { - auto prepared_source_step - = std::make_unique(Pipe(std::make_shared(prepared_input)), context); - query_plan.addStep(std::move(prepared_source_step)); - } - else if (prepared_pipe) + if (prepared_pipe) { auto prepared_source_step = std::make_unique(std::move(*prepared_pipe), context); query_plan.addStep(std::move(prepared_source_step)); @@ -1580,7 +1557,7 @@ void InterpreterSelectQuery::addPrewhereAliasActions() { if (!expressions.prewhere_info) { - const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere(); + const bool does_storage_support_prewhere = !input_pipe && storage && storage->supportsPrewhere(); if (does_storage_support_prewhere && shouldMoveToPrewhere()) { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 99c95a8d624..21e15bc74bb 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -108,7 +108,6 @@ private: InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, - const BlockInputStreamPtr & input_, std::optional input_pipe, const StoragePtr & storage_, const SelectQueryOptions &, @@ -122,7 +121,7 @@ private: Block getSampleBlockImpl(); - void executeImpl(QueryPlan & query_plan, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe); + void executeImpl(QueryPlan & query_plan, std::optional prepared_pipe); /// Different stages of query execution. @@ -198,7 +197,6 @@ private: TableLockHolder table_lock; /// Used when we read from prepared input, not table or subquery. - BlockInputStreamPtr input; std::optional input_pipe; Poco::Logger * log; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 9cfc43b92c5..1fc551334e2 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB @@ -592,9 +592,10 @@ void MergeJoin::mergeInMemoryRightBlocks() builder.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, 0, nullptr, 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); - auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); - while (Block block = sorted_input.read()) + Block block; + while (executor.pull(block)) { if (!block.rows()) continue; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2c12c4a6879..3d0813579ce 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp deleted file mode 100644 index bdfbbc2874e..00000000000 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp +++ /dev/null @@ -1,124 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipeline pipeline_) - : pipeline(std::make_unique(std::move(pipeline_))) -{ -} - -PipelineExecutingBlockInputStream::~PipelineExecutingBlockInputStream() = default; - -Block PipelineExecutingBlockInputStream::getHeader() const -{ - if (executor) - return executor->getHeader(); - - if (async_executor) - return async_executor->getHeader(); - - return pipeline->getHeader(); -} - -void PipelineExecutingBlockInputStream::createExecutor() -{ - if (pipeline->getNumThreads() > 1) - async_executor = std::make_unique(*pipeline); - else - executor = std::make_unique(*pipeline); - - is_execution_started = true; -} - -void PipelineExecutingBlockInputStream::readPrefixImpl() -{ - createExecutor(); -} - -Block PipelineExecutingBlockInputStream::readImpl() -{ - if (!is_execution_started) - createExecutor(); - - Block block; - bool can_continue = true; - while (can_continue) - { - if (executor) - can_continue = executor->pull(block); - else - can_continue = async_executor->pull(block); - - if (block) - return block; - } - - totals = executor ? executor->getTotalsBlock() - : async_executor->getTotalsBlock(); - - extremes = executor ? executor->getExtremesBlock() - : async_executor->getExtremesBlock(); - - return {}; -} - -inline static void throwIfExecutionStarted(bool is_execution_started, const char * method) -{ - if (is_execution_started) - throw Exception(String("Cannot call ") + method + - " for PipelineExecutingBlockInputStream because execution was started", - ErrorCodes::LOGICAL_ERROR); -} - -void PipelineExecutingBlockInputStream::cancel(bool kill) -{ - IBlockInputStream::cancel(kill); - - if (is_execution_started) - { - executor ? executor->cancel() - : async_executor->cancel(); - } -} - -void PipelineExecutingBlockInputStream::setProgressCallback(const ProgressCallback & callback) -{ - throwIfExecutionStarted(is_execution_started, "setProgressCallback"); - pipeline->setProgressCallback(callback); -} - -void PipelineExecutingBlockInputStream::setProcessListElement(QueryStatus * elem) -{ - throwIfExecutionStarted(is_execution_started, "setProcessListElement"); - IBlockInputStream::setProcessListElement(elem); - pipeline->setProcessListElement(elem); -} - -void PipelineExecutingBlockInputStream::setLimits(const StreamLocalLimits & limits_) -{ - throwIfExecutionStarted(is_execution_started, "setLimits"); - - if (limits_.mode == LimitsMode::LIMITS_TOTAL) - throw Exception("Total limits are not supported by PipelineExecutingBlockInputStream", - ErrorCodes::LOGICAL_ERROR); - - /// Local limits may be checked by IBlockInputStream itself. - IBlockInputStream::setLimits(limits_); -} - -void PipelineExecutingBlockInputStream::setQuota(const std::shared_ptr &) -{ - throw Exception("Quota is not supported by PipelineExecutingBlockInputStream", - ErrorCodes::LOGICAL_ERROR); -} - -} diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.h b/src/Processors/Executors/PipelineExecutingBlockInputStream.h deleted file mode 100644 index 68497938ad4..00000000000 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -class QueryPipeline; -class PullingAsyncPipelineExecutor; -class PullingPipelineExecutor; - -/// Implement IBlockInputStream from QueryPipeline. -/// It's a temporary wrapper. -class PipelineExecutingBlockInputStream : public IBlockInputStream -{ -public: - explicit PipelineExecutingBlockInputStream(QueryPipeline pipeline_); - ~PipelineExecutingBlockInputStream() override; - - String getName() const override { return "PipelineExecuting"; } - Block getHeader() const override; - - void cancel(bool kill) override; - - /// Implement IBlockInputStream methods via QueryPipeline. - void setProgressCallback(const ProgressCallback & callback) final; - void setProcessListElement(QueryStatus * elem) final; - void setLimits(const StreamLocalLimits & limits_) final; - void setQuota(const std::shared_ptr & quota_) final; - -protected: - void readPrefixImpl() override; - Block readImpl() override; - -private: - std::unique_ptr pipeline; - /// One of executors is used. - std::unique_ptr executor; /// for single thread. - std::unique_ptr async_executor; /// for many threads. - bool is_execution_started = false; - - void createExecutor(); -}; - -} diff --git a/src/Processors/Formats/InputStreamFromInputFormat.h b/src/Processors/Formats/InputStreamFromInputFormat.h deleted file mode 100644 index 339f559ac9b..00000000000 --- a/src/Processors/Formats/InputStreamFromInputFormat.h +++ /dev/null @@ -1,67 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class InputStreamFromInputFormat : public IBlockInputStream -{ -public: - explicit InputStreamFromInputFormat(InputFormatPtr input_format_) - : input_format(std::move(input_format_)) - , port(input_format->getPort().getHeader(), input_format.get()) - { - connect(input_format->getPort(), port); - port.setNeeded(); - } - - String getName() const override { return input_format->getName(); } - Block getHeader() const override { return input_format->getPort().getHeader(); } - - void cancel(bool kill) override - { - input_format->cancel(); - IBlockInputStream::cancel(kill); - } - - const BlockMissingValues & getMissingValues() const override { return input_format->getMissingValues(); } - -protected: - - Block readImpl() override - { - while (true) - { - auto status = input_format->prepare(); - - switch (status) - { - case IProcessor::Status::Ready: - input_format->work(); - break; - - case IProcessor::Status::Finished: - return {}; - - case IProcessor::Status::PortFull: - return input_format->getPort().getHeader().cloneWithColumns(port.pull().detachColumns()); - - case IProcessor::Status::NeedData: - case IProcessor::Status::Async: - case IProcessor::Status::ExpandPipeline: - throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); - } - } - } - -private: - InputFormatPtr input_format; - InputPort port; -}; - -} diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index ec288484ca3..acf46b95346 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include @@ -9,6 +8,7 @@ #include #include #include +#include #include #include @@ -164,13 +164,7 @@ Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, Output Pipe::Pipe(ProcessorPtr source) { - if (auto * source_from_input_stream = typeid_cast(source.get())) - { - /// Special case for SourceFromInputStream. Will remove it later. - totals_port = source_from_input_stream->getTotalsPort(); - extremes_port = source_from_input_stream->getExtremesPort(); - } - else if (source->getOutputs().size() != 1) + if (source->getOutputs().size() != 1) checkSource(*source); if (collected_processors) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index a76f99c9e00..98ac81f7217 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -121,7 +120,6 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) /// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor. std::vector limits; - std::vector sources; std::vector remote_sources; std::unordered_set visited; @@ -151,9 +149,6 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) limits.emplace_back(limit); } - if (auto * source = typeid_cast(processor)) - sources.emplace_back(source); - if (auto * source = typeid_cast(processor)) remote_sources.emplace_back(source); } @@ -186,16 +181,13 @@ static void initRowsBeforeLimit(IOutputFormat * output_format) } } - if (!rows_before_limit_at_least && (!limits.empty() || !sources.empty() || !remote_sources.empty())) + if (!rows_before_limit_at_least && (!limits.empty() || !remote_sources.empty())) { rows_before_limit_at_least = std::make_shared(); for (auto & limit : limits) limit->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : sources) - source->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : remote_sources) source->setRowsBeforeLimitCounter(rows_before_limit_at_least); } diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/Processors/QueryPipelineBuilder.cpp index 8ed413166da..08a568b48f1 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -468,7 +467,6 @@ void QueryPipelineBuilder::initRowsBeforeLimit() /// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor. std::vector limits; - std::vector sources; std::vector remote_sources; std::unordered_set visited; @@ -498,9 +496,6 @@ void QueryPipelineBuilder::initRowsBeforeLimit() limits.emplace_back(limit); } - if (auto * source = typeid_cast(processor)) - sources.emplace_back(source); - if (auto * source = typeid_cast(processor)) remote_sources.emplace_back(source); } @@ -533,16 +528,13 @@ void QueryPipelineBuilder::initRowsBeforeLimit() } } - if (!rows_before_limit_at_least && (!limits.empty() || !sources.empty() || !remote_sources.empty())) + if (!rows_before_limit_at_least && (!limits.empty() || !remote_sources.empty())) { rows_before_limit_at_least = std::make_shared(); for (auto & limit : limits) limit->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : sources) - source->setRowsBeforeLimitCounter(rows_before_limit_at_least); - for (auto & source : remote_sources) source->setRowsBeforeLimitCounter(rows_before_limit_at_least); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index b4ff1a1281c..d1b9c2cad63 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Sources/SourceFromInputStream.cpp b/src/Processors/Sources/SourceFromInputStream.cpp deleted file mode 100644 index 57e449370a5..00000000000 --- a/src/Processors/Sources/SourceFromInputStream.cpp +++ /dev/null @@ -1,195 +0,0 @@ -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -SourceFromInputStream::SourceFromInputStream(BlockInputStreamPtr stream_, bool force_add_aggregating_info_) - : ISourceWithProgress(stream_->getHeader()) - , force_add_aggregating_info(force_add_aggregating_info_) - , stream(std::move(stream_)) -{ - init(); -} - -void SourceFromInputStream::init() -{ - const auto & sample = getPort().getHeader(); - for (auto & type : sample.getDataTypes()) - if (typeid_cast(type.get())) - has_aggregate_functions = true; -} - -void SourceFromInputStream::addTotalsPort() -{ - if (totals_port) - throw Exception("Totals port was already added for SourceFromInputStream.", ErrorCodes::LOGICAL_ERROR); - - outputs.emplace_back(outputs.front().getHeader(), this); - totals_port = &outputs.back(); -} - -void SourceFromInputStream::addExtremesPort() -{ - if (extremes_port) - throw Exception("Extremes port was already added for SourceFromInputStream.", ErrorCodes::LOGICAL_ERROR); - - outputs.emplace_back(outputs.front().getHeader(), this); - extremes_port = &outputs.back(); -} - -IProcessor::Status SourceFromInputStream::prepare() -{ - auto status = ISource::prepare(); - - if (status == Status::Finished) - { - is_generating_finished = true; - - /// Read postfix and get totals if needed. - if (!is_stream_finished && !isCancelled()) - return Status::Ready; - - if (totals_port && !totals_port->isFinished()) - { - if (has_totals) - { - if (!totals_port->canPush()) - return Status::PortFull; - - totals_port->push(std::move(totals)); - has_totals = false; - } - - totals_port->finish(); - } - - if (extremes_port && !extremes_port->isFinished()) - { - if (has_extremes) - { - if (!extremes_port->canPush()) - return Status::PortFull; - - extremes_port->push(std::move(extremes)); - has_extremes = false; - } - - extremes_port->finish(); - } - } - - return status; -} - -void SourceFromInputStream::work() -{ - if (!is_generating_finished) - { - try - { - ISource::work(); - } - catch (...) - { - /// Won't read suffix in case of exception. - is_stream_finished = true; - throw; - } - - return; - } - - if (is_stream_finished) - return; - - if (rows_before_limit) - { - const auto & info = stream->getProfileInfo(); - if (info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); - } - - stream->readSuffix(); - - if (auto totals_block = stream->getTotals()) - { - totals.setColumns(totals_block.getColumns(), 1); - has_totals = true; - } - - is_stream_finished = true; -} - -Chunk SourceFromInputStream::generate() -{ - if (is_stream_finished) - return {}; - - if (!is_stream_started) - { - stream->readPrefix(); - is_stream_started = true; - } - - auto block = stream->read(); - if (!block && !isCancelled()) - { - if (rows_before_limit) - { - const auto & info = stream->getProfileInfo(); - if (info.hasAppliedLimit()) - rows_before_limit->add(info.getRowsBeforeLimit()); - } - - stream->readSuffix(); - - if (auto totals_block = stream->getTotals()) - { - if (totals_block.rows() > 0) /// Sometimes we can get empty totals. Skip it. - { - totals.setColumns(totals_block.getColumns(), totals_block.rows()); - has_totals = true; - } - } - - if (auto extremes_block = stream->getExtremes()) - { - if (extremes_block.rows() > 0) /// Sometimes we can get empty extremes. Skip it. - { - extremes.setColumns(extremes_block.getColumns(), extremes_block.rows()); - has_extremes = true; - } - } - - is_stream_finished = true; - return {}; - } - - if (isCancelled()) - return {}; - -#ifndef NDEBUG - assertBlocksHaveEqualStructure(getPort().getHeader(), block, "SourceFromInputStream"); -#endif - - UInt64 num_rows = block.rows(); - Chunk chunk(block.getColumns(), num_rows); - - if (force_add_aggregating_info || has_aggregate_functions) - { - auto info = std::make_shared(); - info->bucket_num = block.info.bucket_num; - info->is_overflows = block.info.is_overflows; - chunk.setChunkInfo(std::move(info)); - } - - return chunk; -} - -} diff --git a/src/Processors/Sources/SourceFromInputStream.h b/src/Processors/Sources/SourceFromInputStream.h deleted file mode 100644 index 9649385909c..00000000000 --- a/src/Processors/Sources/SourceFromInputStream.h +++ /dev/null @@ -1,77 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class IBlockInputStream; -using BlockInputStreamPtr = std::shared_ptr; - -/// Wrapper for IBlockInputStream which implements ISourceWithProgress. -class SourceFromInputStream : public ISourceWithProgress -{ -public: - /// If force_add_aggregating_info is enabled, AggregatedChunkInfo (with bucket number and is_overflows flag) will be added to result chunk. - explicit SourceFromInputStream(BlockInputStreamPtr stream_, bool force_add_aggregating_info_ = false); - String getName() const override { return "SourceFromInputStream"; } - - Status prepare() override; - void work() override; - - Chunk generate() override; - - BlockInputStreamPtr & getStream() { return stream; } - - void addTotalsPort(); - void addExtremesPort(); - - OutputPort * getTotalsPort() const { return totals_port; } - OutputPort * getExtremesPort() const { return extremes_port; } - - void setRowsBeforeLimitCounter(RowsBeforeLimitCounterPtr counter) { rows_before_limit.swap(counter); } - - /// Implementation for methods from ISourceWithProgress. - void setLimits(const StreamLocalLimits & limits_) final { stream->setLimits(limits_); } - void setLeafLimits(const SizeLimits &) final { } - void setQuota(const std::shared_ptr & quota_) final { stream->setQuota(quota_); } - void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); } - void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); } - void addTotalRowsApprox(size_t value) final { stream->addTotalRowsApprox(value); } - - /// Stop reading from stream if output port is finished. - void onUpdatePorts() override - { - if (getPort().isFinished()) - cancel(); - } - -protected: - void onCancel() override { stream->cancel(false); } - -private: - bool has_aggregate_functions = false; - bool force_add_aggregating_info = false; - BlockInputStreamPtr stream; - - RowsBeforeLimitCounterPtr rows_before_limit; - - Chunk totals; - OutputPort * totals_port = nullptr; - bool has_totals = false; - - Chunk extremes; - OutputPort * extremes_port = nullptr; - bool has_extremes = false; - - bool is_generating_finished = false; - bool is_stream_finished = false; - bool is_stream_started = false; - - void init(); -}; - -} diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 1639bc4df4b..2a515fdf3be 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -24,9 +24,6 @@ public: Int32 bucket_num = -1; }; -class IBlockInputStream; -using BlockInputStreamPtr = std::shared_ptr; - using AggregatorList = std::list; using AggregatorListPtr = std::shared_ptr; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 19385e526a7..12558054d81 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include @@ -23,6 +22,9 @@ #include #include #include +#include +#include +#include #include #include @@ -124,12 +126,13 @@ public: auto compression = chooseCompressionMethod(path, compression_method); read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri, path, getContext()->getGlobalContext()->getConfigRef()), compression); auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size); + pipeline = QueryPipeline(std::move(input_format)); - reader = std::make_shared(input_format); - reader->readPrefix(); + reader = std::make_unique(pipeline); } - if (auto res = reader->read()) + Block res; + if (reader->pull(res)) { Columns columns = res.getColumns(); UInt64 num_rows = res.rows(); @@ -153,15 +156,16 @@ public: return Chunk(std::move(columns), num_rows); } - reader->readSuffix(); reader.reset(); + pipeline.reset(); read_buf.reset(); } } private: std::unique_ptr read_buf; - BlockInputStreamPtr reader; + QueryPipeline pipeline; + std::unique_ptr reader; SourcesInfoPtr source_info; String uri; String format; diff --git a/src/Storages/Kafka/KafkaSource.cpp b/src/Storages/Kafka/KafkaSource.cpp index 68fc17a97e5..30c5ef3e6cd 100644 --- a/src/Storages/Kafka/KafkaSource.cpp +++ b/src/Storages/Kafka/KafkaSource.cpp @@ -1,9 +1,7 @@ #include -#include #include #include -#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 903ea81946d..690c9cbd4d0 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -16,7 +16,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 357659b3bbb..e17a3dcf544 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -19,7 +19,6 @@ #include "Processors/Merges/GraphiteRollupSortedTransform.h" #include "Processors/Merges/AggregatingSortedTransform.h" #include "Processors/Merges/VersionedCollapsingTransform.h" -#include "Processors/Executors/PipelineExecutingBlockInputStream.h" #include "DataStreams/TTLBlockInputStream.h" #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 683689a6fa3..f4adee8c259 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -35,7 +35,6 @@ #include #include #include -#include #include #include #include @@ -56,6 +55,7 @@ #include #include #include +#include #include #include @@ -3481,9 +3481,12 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc buf, metadata_snapshot->getPartitionKey().sample_block, local_context->getSettingsRef().max_block_size); - auto input_stream = std::make_shared(input_format); + QueryPipeline pipeline(std::move(input_format)); + PullingPipelineExecutor executor(pipeline); + + Block block; + executor.pull(block); - auto block = input_stream->read(); if (!block || !block.rows()) throw Exception( "Could not parse partition value: `" + partition_ast.fields_str + "`", diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 903f4cd27fc..1ee8423b0d1 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -28,7 +28,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index d939312c0bb..50ab26ec470 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -13,8 +13,6 @@ #include #include #include -#include -#include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b8941fc9d84..6ad29d01ca6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,9 +4,9 @@ #include #include #include +#include #include #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index fdc30919ee7..9e138e9882a 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 5a99be7f38c..ff47866d587 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -7,7 +7,6 @@ #include #include #include -#include #include diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 0cd758cf49d..4e9e2c450b1 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/RabbitMQ/RabbitMQSource.cpp b/src/Storages/RabbitMQ/RabbitMQSource.cpp index 34edd06d3e2..b954ad3ab23 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSource.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0944a8f12d5..2b4f5e4a276 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 1ab168f772f..3a37e57e008 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -17,8 +17,8 @@ #include #include -#include #include +#include #include #include diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9747ea2dbf6..b774622d569 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -24,8 +24,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index c8bc215dd6c..e305d4c6183 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b9c15e19c33..df92b270542 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -59,7 +59,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index ec0bd5e5840..9aa5689aa66 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -34,8 +34,8 @@ #include #include #include -#include #include +#include #include #include #include diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index d707d7a6cdf..1138794adb0 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index d2493ff7c43..5a9e8fc2461 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -21,7 +21,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ea42b48cace..6492c9e07c0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -383,7 +383,7 @@ Pipe StorageMerge::createSources( { pipe = QueryPipelineBuilder::getPipe(InterpreterSelectQuery( modified_query_info.query, modified_context, - std::make_shared(header), + Pipe(std::make_shared(header)), SelectQueryOptions(processed_stage).analyze()).buildQueryPipeline()); pipe.addInterpreterContext(modified_context); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 5521b9de39c..eeb5b107b54 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 001684c076d..4264be9dbc2 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index caefa097c3a..709c9dc4a63 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -52,7 +53,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e4682efeaad..e6d41a53bfc 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -25,9 +25,7 @@ #include #include #include -#include #include -#include #include "Processors/Sources/SourceWithProgress.h" #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 32660cb1b1f..d7eef35e60b 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -13,7 +13,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 75ad2761362..79f1d568057 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -13,7 +13,7 @@ #include #include -#include +#include #include #include diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 160fc3c2468..7bd8ad2e740 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -21,7 +21,6 @@ #include #include #include -#include #include "registerTableFunctions.h" From ac1c90e955ac351f7ab7a90778678adb6f24e230 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Wed, 13 Oct 2021 12:48:35 -0600 Subject: [PATCH 599/950] Add careers to nav --- website/templates/global/nav.html | 3 +++ 1 file changed, 3 insertions(+) diff --git a/website/templates/global/nav.html b/website/templates/global/nav.html index 0077ce0496b..7e120c99126 100644 --- a/website/templates/global/nav.html +++ b/website/templates/global/nav.html @@ -47,6 +47,9 @@ + From da1db34c4975f101c47cdce5eb14468024fbb7f3 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Wed, 13 Oct 2021 12:52:33 -0600 Subject: [PATCH 600/950] Update yandex logo --- website/images/logos/logo-yandex.png | Bin 4268 -> 4152 bytes website/templates/index/success.html | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/website/images/logos/logo-yandex.png b/website/images/logos/logo-yandex.png index 900fd13a6838a8f8a3279a11b82b15ac005c1054..f84f4c1b58d7ebf2b02b77d9a9251ae46706e7cf 100644 GIT binary patch literal 4152 zcma)9X*d+#zaNB*Mq?*S_KX;0nK8tWb?o~#$QpjgAWM-jGK{T4wzBWCmqA%)ERm%w z31RGnLUy@&{?EPl&Hdkd&x`N#`F@sjzR!7ap6A3G8EAok*MI;30HmX>ZUO*M5H7Ii zWvYvMl-j^}K|qZ3%rq|A|9zd`{I~k43eUgQ{wLu~`TzEx{pX&i{}1zj9da?rzlJ$S zaGopk{i{7y;XX(H~FnLc`hP1o!+jd@S!-aLoG*o$$^ud5?`tW`xA zb%r`zfcxvg004uXj=HK@@VCWpx_*&JmgxBxj^!m#etK>5K)&cwD_m*son69-gZgq< zDNCc4!ePQ1UAXa{cl$?eP)?XfEk&JuxRc~zQlRFwLo>E1nqCt+_IGSIwSuQ*@L|Z8 zD0nDW+1xM94Lm5I4?D%eW}Bn5akb$OjpMEP)mY5;qsQefQm=fOw@U?jyxvQEP|GlY zn`wTRw)9@S?5Z}u{|}yr(#_Ljlln8ZGf%os_j>E~&@1?6lm__9L4QjVOb&F3B2<@)B;sv<=gYSS{WnF1aU|eggi#Epn|2)+KX^F3 zeJ(s|Y}TChL$kasTp@Ab3>wtZPqVdQE1sjw^qe*lHlqN{f_3>S^@Y?heB1vy75~XM zw1OS!=}AIO%zQMf5As{SMm&o0t+$;ymUnElZlF4>R(f2O_;ct>K*u=4%mU4yKCk-~ z9$CriTl+xyQ;~ERm;hCH?U|I!J~F{Sc(`LWemUrL1LxoQrngh2Z`=I-nQ4e0e#tJc z0dfU`~o4(2F>9pV9`yX#tfmp z9^A{6$lW?xx&G~8e(zx<7i0NX-vIJ3Rq2p)#-^Zl=aw^Ny$%Ay5TZF|ajWTUnmR4{ zN+qB_x}3CAk9Ou{H_~-3XP$^s#WGEq;_hYlsH3zzh#;=gcB{{urcO_5hc;jK>Calc z%v8aLr8m&aPjs>b6pLY^oOoRYu9D%h>YIv@a730Kn}vev$0&ekmW!3NM%fXoPHXh8 zY;T$rYkj6B`nwv)4mbale6ah@V*2?&F>HlBx$aWroxpYLayDH*Mj+z=XR&LwMC_OP zO+bI5C+v3&0s}KjG8Y<(Okte)!0DH^viHiiVLcogmS)~QYNn2P$kzUBBR4L_EW3u@ z1##PIo&|DJ(>V$@V1_t_@(a)2pKm5>4jvO(pX7bOd%9pgFcjx&FsS-ON~4Q3I^l*6 zp2dB?Zy)cLJcnm68)v_U7_kvFG0W1$LPB{U0ibOIM!0Hj5$HN^AA-n1;Q$J?QSLz zzRUZmpNwftcrP)^NjoQI5Hp1~8UFEo!`2UT%GV5{?ED&)ZTy&U zP~+tx*~EW6yg91>_K)fKOBgkmXVvyJp~|*;36VtZgyM+@e)K+&w&fKr5eHxqtH7YU zge7_UIMQ@NdW;-c&wSpaN1}?VM?J^e?UUZ`pf{uQSnkPC4P~p8AdIs`hnG)%I)$si zZAIR-?{iH)UBWDBfJ6g1AqubH3!|+jx08}T3!aY^TM0=(mmiQB77N2;MN0;m>FK6k zRwT<{W$h&<;g zCGGar$ht74qW3;oV9nRxu<->32Js~d@oe1Hvb+lJDTg9QdhL7n*&9*7P zTYNo9#ua`hcdo14^JqVXJt#X$9c@$1LXz{EgQqc#(U$7F_S6x0wgL*^!aI9N z1w37KmeVtnS*MVW8?)6B%$6TBkzjdSp;)fTb~3QUWGJ65icTraIJ@6|FmvFU^@+`E zZ|9$Gd%Po#kDWGbu5U4Y_b2B=v=`l>;<2`Xp;$hjzHy6Sv3$a1IqP^KXHHT8zZ)=TZVq;Ov*Na#AA1_Xnjt3 z%9mK2%7&afcPQVX-61>(lWRkZdO4u=Z$J~VEQhz)i4XcFpp-+;@nyz9yOggHPgrW- zpF70;!x#Q0#KsLnJyBC2aYkW0XH|e~rl#~G?u4_jOK0;M-73+HhD!NvdG`3M_GSc-M2a|SL#rFjw{xb0!<5&v z1v;t-ariuodd32R1dXz=f8?M_UW~|iv*(|)DT%8leYDnozgfskVi<{qoh&j3DRp%- zYgaX$?Ta45P~G%XTX@v2MFRKIxhuw5p-GTiPq7&poy}F45r7&n4XPb&Gs)$cC1@Mr zH_C`_?kI;s6Uc6*NCOR@kt7#acz2z@e=Vfo zT*}aZwh~{#&tK%w^Pm#95i?yrsv+(W+oXtUk}dlYR{6y;R@YZrj7qM6PzAz7GcaK< zm5Mp(JWnz4*uFLMt>LToU5><0=H&cNw`y)^V7!fGjU-put9F3d^p!82K^@+2 zc+Jrf^dVeQL3Y}}z7T5t6u3e%K{9XLcWs}M^y{Q^<#<5H6gAl2E3DYyz9mI|?aD`t zT$JWUP21c^RE|Z=4S^HLN(jfQc6(Wt8Tp2uvZvbqsrE!?yuW*U1E^K@6IsmF@wz5* zl7Xxy3?)9yU5y*ddx%kphBy5*wttFP=9gkX9TIeY#7kr<;s}k7NNa4F+0HQ zB(snn@Quwm&5oEiLeTAXLq!PetADocJ5Y%^=5_YK2tG@UbeoQMiZeh6BgL$?N~xAG zR!Ip+ElQUC9CyOucFRt&nGiE@*Wst)H84qdEU8u?BLl4JGt+jk{k51N&t!kyjvF=~ z%p`!E)0V546zFv!%QRVg^bI#UTa4ge^uJkEb#5X#2qIj2FOy9WnA7gYo~sIVa6(b$ z?owJ|O>J1r>!7UTz)`QbN7sH@3Dm2pjikY^LJa9or_pQ^S5yGz<12a^<#)z7rZkFT z*hs0#;4DC%i5kil+-MZH@PeWd*YsFCnSe^Oy~o=A@>J*LO<`VUSV7#;a*`Fx#6sHH zgF1s*zB~3{OG|uKeTgvdsm`wfjVO3oQea~aMdH4Okfvw*Q3jns(fm6P!)Y+*$Jmb4 zHsK7-#54)asea^==0@W?_<`Vi0`R<`z-HqZt{heuf5f_dfR#vpoBtfiZycYa(7A85V8M?TwUG)8C zVaktKMb+zw{@$pG6`sAG`Wp;`#z|fAi`UARGx~IrHP=l4A%F82z4cDihg-3b5O~i4 z^S~1NyY=quLswmHS)+b6@PQN-9(HnUWy40hV447TzEO&>hMeG{9BHG#ed?v~rejF$X3T5ZwS%4 zwq`Vjg@v~7h^q74)YH5#=vw)4_$>$nTTqcM*HWT1&s^-F)D<79)-HK;cSw43)5JJY z6%(!b+23dX=xaIrh_#F`@WP4e1zc4ABS*UHO!#6P_Me{N2|V*!#%n>Dq?uoMD|T89 z+A~aCD500aA^)lrN4=b)%riyT60}gjw|!#A&sw}vkO567pBDtjB}Qa4ky%_b=ncHd zs#T_vw%xq55M67&X8pBJt|uQ37}60G&hk3pnRy_AD{%SR_`;Hzq6(l;%J&QIL+j|9c)}pB0+qzs9#H}~1F}Hqsy;?+5Y~bbuyeR>YHDiu`1lYI5P*Py{QUgc+1UsP z2=?~&@$vC2EG+Hq?ez5Y=;-J-H#Z+2AEBY4@9*#b1_rUQv07SMF)=YbJUkQ>6#x18 zDk>`fIy(RB>)_zv!NI|xpr9ruCg$em&(F^r8yo))4yvlE|AmD!Gc*4&G5`1X|HZ|{ z#l`dU^W5CrFE1~8dV0*v%;MtWzP`Ty2ngQZ-qzOExw*MtUtc35BmevRwY9aat*t^r zLKqkr|MTCd?=jWuPq?41Ab#--7QBg}vOaIo^|GBxJpPz_`h;njrPft((92}gSoQ#Z&Y;0`* zWMn@-KZJyY|KsET+S=II*#CTd|F5tAot^)dmj88i|7vRgKtTU3Ez!}@|D&UsnVD~I zZ~sY2|0gH^zP|sBjsI_N|6E-ER#yKaBLD5}|K8sJ%F6%3!vCkITwGlLPf!1mk^e+Q zn3$M~+G&me01m)OL_t(|+TEM^U)o3%!0AG`N+b}FLlm{FD2GTbateZoiWNoizUpnY z9zCo_Yq#Cr?f%gV^Cl!Dm=$e5Tl;?T!(=iu-+6D|ym=WVSCF=MkCD8Q{9ZEm=pm99 zr;^`GR-XQW-)N!!y`i0kd}kmR?L{lm=?Snr!B{gT@x_qwsj zFRAeHMQ;hL^Yo`$qLiSK5b_^pX)D7ZW7QSXe+riP^1)BFL^>Mrl*DcNVLtD;6UpY$ zTy>2;evK^g>)cPZMBTa)$z8>dYVkb`{Ex*_>L|8A33l>ho*lKoR`i^pSC z#{Q%iJ=}c9dv*)Jh8}!xM1z0%slQ5`2HhXN7~G3CId3a?3#$D5f&b2}0FyrX6FdBsN|p6L`;N-*KM*Xj zTLC5+dJbfNKborm?*6)o<3zzOc00iOKZEn@{Z`>V`uATqqHyY&FtEQJVDe90;PIbd z+~cmjM>tOWB=m{515EPaGq{iAr(E5<2cOvZS|XHN0@gkJwGsT{3fB@L)cUil+AFsM zO!@@d;_)+1{CfoZ_S**%q1+NMN!uxay?)H@WW#S@6MJ5D6T2l~{U08J-EEFZ%W>jk zu*9boAtXhcdTb@3NkXs*5F`k=B!Vv0Gmw=0GDCqHCYqEUTnl;u25!6XXDPHZLQf=l3lf2?a611)>1u{wRx(^CKqVS z(5{&_dS%5LbzYYxoT&;X*DYv5URAM{UTL+2XWcAVyEask=CwE{b28tI3TK0C3m?9B zLX`309yd;GEJq`hTQR$6y~7n*vLj&PjJ1AJiHcKZbs(QZ{7AfuSwU8$XV<3m{0`0`?o8z=T2?{{N$x00n2{f39+ zH1g;Wy!+FAG{_;Gc6-r=6{aY#eXu<#Em2AUW`yx@M1Pj3Tseyt*?)t@8&rb+JdExQ z5op3SN=uogkU=H-yu0#i5x9@P(4q3)Fm@)zUjQDor-}_1-}HuZxdO5rP`g^n1Cyy7 zr49xyQyay<0I+noVtt)8Gefp!av5uNFvF+|HCOCpD#MtqHh>jGnb(yF!@*1~l?Ag^ zdgzLL)DgIkSEoZIFZuZ)xc~irJecT?zcgm6CAYJ%5FTxoVO?m06b}sMHng;OcRS?H7wxCIq8Ae&``(S(?yJJFw;XOpG<3K_i0!5?kUwDO5ki z@xn+)?;1g6PfE`IRWC12v}vb{T*ir@%|z3*E#j5Q(Abs(v=B_DO2yR0dZcd$54nI8 z3!t0l;ZaZG=Fg^};W}VcPrw68^PHV#>H0n#gQ~!{> z6ylrSa;mL+xDl(5wV$movZa9pvv`Svqyt9hvkAc*dvVYZncW3`%L!FhSPa{tKdgrR zRZcf7^EGCQgj9%R+aN`b#0Sb>8jv$?!Ja}*>vgCvLkx6kiOcy3Mx6A0SInZ~&$Ha1 zIK80Fs#)tLSCcT7?I0gazdI0QEYCTqN4@*AWw5?5#vhH8hu|6EgQ@3HV}r{Pc#&{U zqk%hEqaL4vmjU>gjH~^?7TlF{n!PrHHx}ea9+Y!Uk2pq{D8eu@gS=J7#29sA9dtBtQfgL_ zL>f(Qk%Gg^VJF~RbRWHgeEt#Vd`6O$Dvo!+STq5?yKmMJ?0Uj3+c-|(TI%A=BgrR3 zg5aKP4lHki87DCduGm}NU<_L=s3e&#j?9+!mtK%fc!ZJg$A>tcU-c`>|kTqJRpNQ&CC`OoJot7-w$R{VW)W zX&%|4%wwwwD;kPvpT-Cv(?gD`2xW}DY z4F=<^qqAvL&jAOPX+Rdr*aH^q?S=3;X**eWDOD=HUN4i$t^=l?hu^6!u`HE5fg}3~ zC*8vc&49IY97=HK4lAP#&`Gsx9p8H~SMT#VfwmG88b5@hAlCqMZ|l)-mro3gXb&-m zA3G}+d~h4*?7@ar$d-UXdM?6u%CAQ@bpGC~vA2aE0jn4?J)tPgjj@>*F@ z>hTuf zm&Zv%e|ECDj|%af>lrRb%kMW3L(qlOaG%;L4bg|YvTE*TkYIaWy6lqTmCGYc?nj2H%}4!Rj~apeJp@bd$)#Q@9EkjU#1Q^O9X4Q+-jc z^WgaVy>@A>U}QUR)onNl$&J8vYXQ^e<0heSuc6w)*j3_FX>DNa`wky{vsf{>bEP#l z-~jtLGfTuJI|>sHN1%XGEB5zA-@@zGhBmLyaHH01F;me3bA4hVA42tJQ;3u z|MT6wD*qo~gcWl79&nA;HL^CadJNGLzA0E#3t*|$t3)1pO;D7Jj-jAFv`&QmC~lZ> z%WDF2k26u`O~Kp?80;T0Fe=i_9$Q4j7FMEsl26k91ynq*t_dtU0R5g1+!RdRfsZYB z0i);c^arJ;i3r3H-ds-& zV736WLCbFj2HzbaFU?H>Cir0W*bk@d!Aspo*pC`IuZN9fLJz@Fgt|Hyp}PZO>?6ur zY=9zvLrq{r45u>&+c{f~Z-WO$d%i&8$(XNjbEG>#F?R}@l}Pn5s9?&1TN>2RHH1`A zupjuoi2ZgLid<(kfyomsdJIuM$y4sIp9e-bx9h-UM^9*u=_r~y7}ZAx5xFCdS8vpX z8dkNKo6MSuE!)VGRRG_r4-iUWKCdf_S&V@Q3#cEPXGjfF*jTsdVER5W1@hc zu%&Y@E8YO{bs5(P$tW4GUR8U~)xij?=R5jbENU{DPbOMrvTK5QCYb9e%O~}Q9+Pu5 z2MAm+`~?$ql2q7=c}0WAx1rJ(O_)R&x(s@2W0vWRI>E_EZS?A3Qgsraz*|GR_DsQ_ zl3g3j6!{_pvO0fZ+1_GtLDM`iDv~r}vQ~e~RBpM&?<%#g4BJqYH$I4NldOB#3pB|T z+|0@LTs<{lx{E9Wy=ax8NLrP$qCto0+Q204Jezf`^TH*S8JCI$1F42Miix%>chZ18Q}gyWP=r-u1H~C@$f!;M7jVlR(j&T=yVQB z`nXnSz{%6WCACmv&mRAYMEL!g3~nv?T~_a)Hx=652^geAB-v7waE!TqwlLjNk|i~( zvHGSADznD`3=8cZuosM}Vq4ynf26}r5$3(p%k#WsuIZ%dykJC~T3sQ3CLh2zO8XXD zkR<4e`AY7h(h(t;bZBS{$-OGO9BEc;?P*3d+c6#5Oo$#sQA~7sf(;-P_j?*HqXk!e z6Y0C>BbxOEg_=Y=BdSFRZ`eB1tCJ%mn($Vi$5u8m+?9=YCAbGl4aC2Nb+V|!Hv8@X O0000 - +
    From 86e8f332719d094e49ee3bf02cdf275f5b29d716 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 13 Oct 2021 16:48:56 -0300 Subject: [PATCH 601/950] Doc. Fix markup --- docs/ru/sql-reference/statements/create/view.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index 7ebb154d6b6..551c0265114 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -68,8 +68,8 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na ## LIVE-представления [экспериментальный функционал] {#live-view} !!! important "Важно" - Представления `LIVE VIEW` являются экспериментальной возможностью. Их использование может повлечь потерю совместимости в будущих версиях. - Чтобы использовать `LIVE VIEW` и запросы `WATCH`, включите настройку [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view). + Представления `LIVE VIEW` являются экспериментальной возможностью. Их использование может повлечь потерю совместимости в будущих версиях. + Чтобы использовать `LIVE VIEW` и запросы `WATCH`, включите настройку [allow_experimental_live_view](../../../operations/settings/settings.md#allow-experimental-live-view). ```sql CREATE LIVE VIEW [IF NOT EXISTS] [db.]table_name [WITH [TIMEOUT [value_in_sec] [AND]] [REFRESH [value_in_sec]]] AS SELECT ... From 62c5c48a3f221b3e7fd21ca4747d4a0c63055263 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 00:02:27 +0300 Subject: [PATCH 602/950] Updated tests --- .../0_stateless/02096_sql_user_defined_function_alias.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql b/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql index 767d5945ee8..70e6572629e 100644 --- a/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql +++ b/tests/queries/0_stateless/02096_sql_user_defined_function_alias.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + CREATE FUNCTION 02096_test_function AS x -> x + 1; DESCRIBE (SELECT 02096_test_function(1) AS a); -DROP FUNCTION 02096_test_function; \ No newline at end of file +DROP FUNCTION 02096_test_function; From 66f2163b60f765ebe6e95172fb34dec40808e52d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 7 Oct 2021 22:39:46 +0300 Subject: [PATCH 603/950] Added function tokens --- src/Functions/ngrams.cpp | 126 ------------- src/Functions/registerFunctions.cpp | 4 +- src/Functions/tokenExtractors.cpp | 166 ++++++++++++++++++ .../queries/0_stateless/2028_tokens.reference | 8 + tests/queries/0_stateless/2028_tokens.sql | 9 + 5 files changed, 185 insertions(+), 128 deletions(-) delete mode 100644 src/Functions/ngrams.cpp create mode 100644 src/Functions/tokenExtractors.cpp create mode 100644 tests/queries/0_stateless/2028_tokens.reference create mode 100644 tests/queries/0_stateless/2028_tokens.sql diff --git a/src/Functions/ngrams.cpp b/src/Functions/ngrams.cpp deleted file mode 100644 index c5ce65537cb..00000000000 --- a/src/Functions/ngrams.cpp +++ /dev/null @@ -1,126 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -class FunctionNgrams : public IFunction -{ -public: - - static constexpr auto name = "ngrams"; - - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 2; } - bool isVariadic() const override { return false; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } - - bool useDefaultImplementationForNulls() const override { return true; } - bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - auto ngram_input_argument_type = WhichDataType(arguments[0].type); - if (!ngram_input_argument_type.isStringOrFixedString()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} second argument type should be String or FixedString. Actual {}", - getName(), - arguments[0].type->getName()); - - const auto & column_with_type = arguments[1]; - const auto & ngram_argument_column = arguments[1].column; - auto ngram_argument_type = WhichDataType(column_with_type.type); - - if (!ngram_argument_type.isNativeUInt() || !ngram_argument_column || !isColumnConst(*ngram_argument_column)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} second argument type should be constant UInt. Actual {}", - getName(), - arguments[1].type->getName()); - - Field ngram_argument_value; - ngram_argument_column->get(0, ngram_argument_value); - auto ngram_value = ngram_argument_value.safeGet(); - - return std::make_shared(std::make_shared(ngram_value)); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override - { - Field ngram_argument_value; - arguments[1].column->get(0, ngram_argument_value); - auto ngram_value = ngram_argument_value.safeGet(); - - NgramTokenExtractor extractor(ngram_value); - - auto result_column_fixed_string = ColumnFixedString::create(ngram_value); - auto column_offsets = ColumnArray::ColumnOffsets::create(); - - auto input_column = arguments[0].column; - if (const auto * column_string = checkAndGetColumn(input_column.get())) - executeImpl(extractor, *column_string, *result_column_fixed_string, *column_offsets); - else if (const auto * column_fixed_string = checkAndGetColumn(input_column.get())) - executeImpl(extractor, *column_fixed_string, *result_column_fixed_string, *column_offsets); - - return ColumnArray::create(std::move(result_column_fixed_string), std::move(column_offsets)); - } - -private: - - template - inline void executeImpl(const NgramTokenExtractor & extractor, StringColumnType & input_data_column, ColumnFixedString & result_data_column, ColumnArray::ColumnOffsets & offsets_column) const - { - size_t current_tokens_size = 0; - auto & offsets_data = offsets_column.getData(); - - size_t column_size = input_data_column.size(); - offsets_data.resize(column_size); - - for (size_t i = 0; i < column_size; ++i) - { - auto data = input_data_column.getDataAt(i); - - size_t cur = 0; - size_t token_start = 0; - size_t token_length = 0; - - while (cur < data.size && extractor.nextInString(data.data, data.size, &cur, &token_start, &token_length)) - { - result_data_column.insertData(data.data + token_start, token_length); - ++current_tokens_size; - } - - offsets_data[i] = current_tokens_size; - } - } -}; - -void registerFunctionNgrams(FunctionFactory & factory) -{ - factory.registerFunction(); -} - -} - - diff --git a/src/Functions/registerFunctions.cpp b/src/Functions/registerFunctions.cpp index b2f038240aa..94dfcdb4fda 100644 --- a/src/Functions/registerFunctions.cpp +++ b/src/Functions/registerFunctions.cpp @@ -37,7 +37,7 @@ void registerFunctionsStringArray(FunctionFactory &); void registerFunctionsStringSearch(FunctionFactory &); void registerFunctionsStringRegexp(FunctionFactory &); void registerFunctionsStringSimilarity(FunctionFactory &); -void registerFunctionNgrams(FunctionFactory &); +void registerFunctionsStringTokenExtractor(FunctionFactory &); void registerFunctionsURL(FunctionFactory &); void registerFunctionsVisitParam(FunctionFactory &); void registerFunctionsMath(FunctionFactory &); @@ -101,7 +101,7 @@ void registerFunctions() registerFunctionsStringSearch(factory); registerFunctionsStringRegexp(factory); registerFunctionsStringSimilarity(factory); - registerFunctionNgrams(factory); + registerFunctionsStringTokenExtractor(factory); registerFunctionsURL(factory); registerFunctionsVisitParam(factory); registerFunctionsMath(factory); diff --git a/src/Functions/tokenExtractors.cpp b/src/Functions/tokenExtractors.cpp new file mode 100644 index 00000000000..f15cfb4cb62 --- /dev/null +++ b/src/Functions/tokenExtractors.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +enum TokenExtractorStrategy +{ + ngrams, + tokens +}; + +template +class FunctionTokenExtractor : public IFunction +{ +public: + + static constexpr auto name = strategy == ngrams ? "ngrams" : "tokens"; + + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return strategy == ngrams ? 2 : 1; } + bool isVariadic() const override { return false; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return strategy == ngrams ? ColumnNumbers{1} : ColumnNumbers{}; } + + bool useDefaultImplementationForNulls() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + auto ngram_input_argument_type = WhichDataType(arguments[0].type); + if (!ngram_input_argument_type.isStringOrFixedString()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} second argument type should be String or FixedString. Actual {}", + getName(), + arguments[0].type->getName()); + + if constexpr (strategy == ngrams) + { + const auto & column_with_type = arguments[1]; + const auto & ngram_argument_column = arguments[1].column; + auto ngram_argument_type = WhichDataType(column_with_type.type); + + if (!ngram_argument_type.isNativeUInt() || !ngram_argument_column || !isColumnConst(*ngram_argument_column)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Function {} second argument type should be constant UInt. Actual {}", + getName(), + arguments[1].type->getName()); + + Field ngram_argument_value; + ngram_argument_column->get(0, ngram_argument_value); + auto ngram_value = ngram_argument_value.safeGet(); + + return std::make_shared(std::make_shared(ngram_value)); + } + else + { + return std::make_shared(std::make_shared()); + } + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + { + auto column_offsets = ColumnArray::ColumnOffsets::create(); + + if constexpr (strategy == TokenExtractorStrategy::ngrams) + { + Field ngram_argument_value; + arguments[1].column->get(0, ngram_argument_value); + auto ngram_value = ngram_argument_value.safeGet(); + + NgramTokenExtractor extractor(ngram_value); + + auto result_column_fixed_string = ColumnFixedString::create(ngram_value); + + auto input_column = arguments[0].column; + + if (const auto * column_string = checkAndGetColumn(input_column.get())) + executeImpl(extractor, *column_string, *result_column_fixed_string, *column_offsets); + else if (const auto * column_fixed_string = checkAndGetColumn(input_column.get())) + executeImpl(extractor, *column_fixed_string, *result_column_fixed_string, *column_offsets); + + return ColumnArray::create(std::move(result_column_fixed_string), std::move(column_offsets)); + } + else + { + SplitTokenExtractor extractor; + + auto result_column_string = ColumnString::create(); + + auto input_column = arguments[0].column; + + if (const auto * column_string = checkAndGetColumn(input_column.get())) + executeImpl(extractor, *column_string, *result_column_string, *column_offsets); + else if (const auto * column_fixed_string = checkAndGetColumn(input_column.get())) + executeImpl(extractor, *column_fixed_string, *result_column_string, *column_offsets); + + return ColumnArray::create(std::move(result_column_string), std::move(column_offsets)); + } + } + +private: + + template + inline void executeImpl( + const ExtractorType & extractor, + StringColumnType & input_data_column, + ResultStringColumnType & result_data_column, + ColumnArray::ColumnOffsets & offsets_column) const + { + size_t current_tokens_size = 0; + auto & offsets_data = offsets_column.getData(); + + size_t column_size = input_data_column.size(); + offsets_data.resize(column_size); + + for (size_t i = 0; i < column_size; ++i) + { + auto data = input_data_column.getDataAt(i); + + size_t cur = 0; + size_t token_start = 0; + size_t token_length = 0; + + while (cur < data.size && extractor.nextInString(data.data, data.size, &cur, &token_start, &token_length)) + { + result_data_column.insertData(data.data + token_start, token_length); + ++current_tokens_size; + } + + offsets_data[i] = current_tokens_size; + } + } +}; + +void registerFunctionsStringTokenExtractor(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); +} + +} + + diff --git a/tests/queries/0_stateless/2028_tokens.reference b/tests/queries/0_stateless/2028_tokens.reference new file mode 100644 index 00000000000..0c23a7598b6 --- /dev/null +++ b/tests/queries/0_stateless/2028_tokens.reference @@ -0,0 +1,8 @@ +['test'] +['test1','test2','test3'] +['test1','test2','test3','test4'] +['test1','test2','test3','test4'] +['test'] +['test1','test2','test3'] +['test1','test2','test3','test4'] +['test1','test2','test3','test4'] diff --git a/tests/queries/0_stateless/2028_tokens.sql b/tests/queries/0_stateless/2028_tokens.sql new file mode 100644 index 00000000000..835ec140302 --- /dev/null +++ b/tests/queries/0_stateless/2028_tokens.sql @@ -0,0 +1,9 @@ +SELECT tokens('test'); +SELECT tokens('test1, test2, test3'); +SELECT tokens('test1, test2, test3, test4'); +SELECT tokens('test1,;\ test2,;\ test3,;\ test4'); + +SELECT tokens(materialize('test')); +SELECT tokens(materialize('test1, test2, test3')); +SELECT tokens(materialize('test1, test2, test3, test4')); +SELECT tokens(materialize('test1,;\ test2,;\ test3,;\ test4')); From 90bac4c31872b339917fafa80c593be698e8c453 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 11 Oct 2021 11:37:38 +0300 Subject: [PATCH 604/950] Fixed tests --- .../0_stateless/{2028_tokens.reference => 02028_tokens.reference} | 0 tests/queries/0_stateless/{2028_tokens.sql => 02028_tokens.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{2028_tokens.reference => 02028_tokens.reference} (100%) rename tests/queries/0_stateless/{2028_tokens.sql => 02028_tokens.sql} (100%) diff --git a/tests/queries/0_stateless/2028_tokens.reference b/tests/queries/0_stateless/02028_tokens.reference similarity index 100% rename from tests/queries/0_stateless/2028_tokens.reference rename to tests/queries/0_stateless/02028_tokens.reference diff --git a/tests/queries/0_stateless/2028_tokens.sql b/tests/queries/0_stateless/02028_tokens.sql similarity index 100% rename from tests/queries/0_stateless/2028_tokens.sql rename to tests/queries/0_stateless/02028_tokens.sql From e6bca4d9288bccfe32c21250683a99de60cac658 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Oct 2021 12:29:52 +0300 Subject: [PATCH 605/950] Updated test --- tests/queries/0_stateless/02028_tokens.reference | 8 ++++++++ tests/queries/0_stateless/02028_tokens.sql | 12 ++++++++++-- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02028_tokens.reference b/tests/queries/0_stateless/02028_tokens.reference index 0c23a7598b6..04098272263 100644 --- a/tests/queries/0_stateless/02028_tokens.reference +++ b/tests/queries/0_stateless/02028_tokens.reference @@ -2,7 +2,15 @@ ['test1','test2','test3'] ['test1','test2','test3','test4'] ['test1','test2','test3','test4'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] ['test'] ['test1','test2','test3'] ['test1','test2','test3','test4'] ['test1','test2','test3','test4'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] +['ё','ё','జ్ఞ‌ా','本気ですか','ﷺ','ᾂ','ΐ','שּ'] diff --git a/tests/queries/0_stateless/02028_tokens.sql b/tests/queries/0_stateless/02028_tokens.sql index 835ec140302..bbb6c8b79ec 100644 --- a/tests/queries/0_stateless/02028_tokens.sql +++ b/tests/queries/0_stateless/02028_tokens.sql @@ -2,8 +2,16 @@ SELECT tokens('test'); SELECT tokens('test1, test2, test3'); SELECT tokens('test1, test2, test3, test4'); SELECT tokens('test1,;\ test2,;\ test3,;\ test4'); +SELECT tokens('ё ё జ్ఞ‌ా 本気ですか ﷺ ᾂ ΐ שּ'); +SELECT tokens('ё, ё, జ్ఞ‌ా, 本気ですか, ﷺ, ᾂ, ΐ, שּ'); +SELECT tokens('ё, ё, జ్ఞ‌ా, 本気ですか, ﷺ, ᾂ, ΐ, שּ'); +SELECT tokens('ё;\ ё;\ జ్ఞ‌ా;\ 本気ですか;\ ﷺ;\ ᾂ;\ ΐ;\ שּ'); SELECT tokens(materialize('test')); SELECT tokens(materialize('test1, test2, test3')); -SELECT tokens(materialize('test1, test2, test3, test4')); -SELECT tokens(materialize('test1,;\ test2,;\ test3,;\ test4')); +SELECT tokens(materialize('test1, test2, test3, test4')); +SELECT tokens(materialize('test1,;\ test2,;\ test3,;\ test4')); +SELECT tokens(materialize('ё ё జ్ఞ‌ా 本気ですか ﷺ ᾂ ΐ שּ')); +SELECT tokens(materialize('ё, ё, జ్ఞ‌ా, 本気ですか, ﷺ, ᾂ, ΐ, שּ')); +SELECT tokens(materialize('ё, ё, జ్ఞ‌ా, 本気ですか, ﷺ, ᾂ, ΐ, שּ')); +SELECT tokens(materialize('ё;\ ё;\ జ్ఞ‌ా;\ 本気ですか;\ ﷺ;\ ᾂ;\ ΐ;\ שּ')); \ No newline at end of file From 58728fe78f6d9f7ec0a92c0a8ca3f660cc2fbe97 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 00:08:43 +0300 Subject: [PATCH 606/950] Fixed tests --- src/Dictionaries/PolygonDictionary.h | 6 ++-- .../PolygonDictionaryImplementations.cpp | 28 +++++++++++-------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 73b782f7692..00a151d546f 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -52,12 +52,12 @@ public: struct Configuration { - /// Store polygon key column. That will allow to read columns from polygon dictionary. - bool store_polygon_key_column = false; - InputType input_type = InputType::MultiPolygon; PointType point_type = PointType::Array; + + /// Store polygon key column. That will allow to read columns from polygon dictionary. + bool store_polygon_key_column = false; }; IPolygonDictionary( diff --git a/src/Dictionaries/PolygonDictionaryImplementations.cpp b/src/Dictionaries/PolygonDictionaryImplementations.cpp index 93ba67a59e9..4f515586120 100644 --- a/src/Dictionaries/PolygonDictionaryImplementations.cpp +++ b/src/Dictionaries/PolygonDictionaryImplementations.cpp @@ -173,8 +173,6 @@ DictionaryPtr createLayout(const std::string & , throw Exception(ErrorCodes::BAD_ARGUMENTS, "The 'key' should consist of a single attribute for a polygon dictionary"); - IPolygonDictionary::Configuration configuration; - const auto key_type = (*dict_struct.key)[0].type; const auto f64 = std::make_shared(); const auto multi_polygon_array = DataTypeArray(std::make_shared(std::make_shared(std::make_shared(f64)))); @@ -182,25 +180,28 @@ DictionaryPtr createLayout(const std::string & , const auto simple_polygon_array = DataTypeArray(std::make_shared(f64)); const auto simple_polygon_tuple = DataTypeArray(std::make_shared(std::vector{f64, f64})); + IPolygonDictionary::InputType input_type; + IPolygonDictionary::PointType point_type; + if (key_type->equals(multi_polygon_array)) { - configuration.input_type = IPolygonDictionary::InputType::MultiPolygon; - configuration.point_type = IPolygonDictionary::PointType::Array; + input_type = IPolygonDictionary::InputType::MultiPolygon; + point_type = IPolygonDictionary::PointType::Array; } else if (key_type->equals(multi_polygon_tuple)) { - configuration.input_type = IPolygonDictionary::InputType::MultiPolygon; - configuration.point_type = IPolygonDictionary::PointType::Tuple; + input_type = IPolygonDictionary::InputType::MultiPolygon; + point_type = IPolygonDictionary::PointType::Tuple; } else if (key_type->equals(simple_polygon_array)) { - configuration.input_type = IPolygonDictionary::InputType::SimplePolygon; - configuration.point_type = IPolygonDictionary::PointType::Array; + input_type = IPolygonDictionary::InputType::SimplePolygon; + point_type = IPolygonDictionary::PointType::Array; } else if (key_type->equals(simple_polygon_tuple)) { - configuration.input_type = IPolygonDictionary::InputType::SimplePolygon; - configuration.point_type = IPolygonDictionary::PointType::Tuple; + input_type = IPolygonDictionary::InputType::SimplePolygon; + point_type = IPolygonDictionary::PointType::Tuple; } else { @@ -218,7 +219,12 @@ DictionaryPtr createLayout(const std::string & , config.keys(layout_prefix, keys); const auto & dict_prefix = layout_prefix + "." + keys.front(); - configuration.store_polygon_key_column = config.getUInt(dict_prefix + ".store_polygon_key_column", false); + IPolygonDictionary::Configuration configuration + { + .input_type = input_type, + .point_type = point_type, + .store_polygon_key_column = config.getBool(dict_prefix + ".store_polygon_key_column", false) + }; if (dict_struct.range_min || dict_struct.range_max) throw Exception(ErrorCodes::BAD_ARGUMENTS, From e4db862d85fb1da177ecbe6f25941eece94aec56 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 23:38:21 +0300 Subject: [PATCH 607/950] Fix possible data-race between FileChecker and StorageLog/StorageStripeLog --- src/Storages/StorageLog.cpp | 7 ++++--- src/Storages/StorageStripeLog.cpp | 5 +---- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..12bb213bace 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -24,6 +24,7 @@ #include #include "StorageLogSettings.h" #include +#include #include #include @@ -122,9 +123,6 @@ Chunk LogSource::generate() if (rows_read == rows_limit) return {}; - if (storage.file_checker.empty()) - return {}; - /// How many rows to read for the next block. size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read); std::unordered_map caches; @@ -672,6 +670,9 @@ Pipe StorageLog::read( if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + if (file_checker.empty()) + return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); + Pipes pipes; const Marks & marks = getMarksWithRealRowCount(metadata_snapshot); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 2dc2577f245..07c66c3034c 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -98,9 +98,6 @@ public: protected: Chunk generate() override { - if (storage.file_checker.empty()) - return {}; - Block res; start(); @@ -337,7 +334,7 @@ Pipe StorageStripeLog::read( Pipes pipes; String index_file = table_path + "index.mrk"; - if (!disk->exists(index_file)) + if (file_checker.empty() || !disk->exists(index_file)) { return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); } From eba6410fd590e93247fce8b4ec3240942e3c01c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Oct 2021 09:13:00 +0300 Subject: [PATCH 608/950] Fix data-race between LogSink::writeMarks() and LogSource::readData() in StorageLog CI founds [1], TSan report: WARNING: ThreadSanitizer: data race (pid=497) Write of size 8 at 0x7b1c00e4e8d8 by thread T922: 4 DB::LogSink::writeMarks() obj-x86_64-linux-gnu/../src/Storages/StorageLog.cpp:462:72 (clickhouse+0x1665cdaa) 5 DB::LogSink::consume() obj-x86_64-linux-gnu/../src/Storages/StorageLog.cpp:324:5 (clickhouse+0x1665c216) Previous read of size 8 at 0x7b1c00e4e8d8 by thread T711: 1 DB::LogSource::readData()::$_0::operator()(bool) const::'lambda'()::operator()() const obj-x86_64-linux-gnu/../src/Storages/StorageLog.cpp:188:26 (clickhouse+0x16661baf) 9 DB::LogSource::readData() obj-x86_64-linux-gnu/../src/Storages/StorageLog.cpp:204:20 (clickhouse+0x1665bbcb) [1]: https://clickhouse-test-reports.s3.yandex.net/29930/4bc90d1dd7dbd4b8a9b6920d00ca24e8b160358e/stress_test_(thread).html#fail1 Fixes: #29470 v2: rework locking, but this produce deadlock for 01499_log_deadlock test v3: introduce per-file marks mutex --- src/Storages/StorageLog.cpp | 8 +++++++- src/Storages/StorageLog.h | 2 ++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..a86275d4c76 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -185,7 +185,10 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu UInt64 offset = 0; if (!stream_for_prefix && mark_number) + { + std::lock_guard marks_lock(file_it->second.marks_mutex); offset = file_it->second.marks[mark_number].offset; + } auto & data_file_path = file_it->second.data_file_path; auto it = streams.try_emplace(stream_name, storage.disk, data_file_path, offset, read_settings).first; @@ -459,7 +462,10 @@ void LogSink::writeMarks(MarksForColumns && marks) writeIntBinary(mark.second.offset, *marks_stream); size_t column_index = mark.first; - storage.files[storage.column_names_by_idx[column_index]].marks.push_back(mark.second); + + auto & file = storage.files[storage.column_names_by_idx[column_index]]; + std::lock_guard marks_lock(file.marks_mutex); + file.marks.push_back(mark.second); } } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index e06f884bdd1..ada971ecf43 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -82,6 +82,8 @@ private: size_t column_index; String data_file_path; + + std::mutex marks_mutex; Marks marks; }; using Files = std::map; /// file name -> column data From c9b6c2661bf24403af9d22da40d2d2b872dd0792 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 6 Oct 2021 14:02:40 +0300 Subject: [PATCH 609/950] Refactor ConcurrentBoundedQueue --- src/Access/ReplicatedAccessStorage.cpp | 6 +- src/Common/ConcurrentBoundedQueue.h | 263 ++++++++++-------- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 20 +- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- src/Coordination/KeeperDispatcher.cpp | 12 +- src/Coordination/ThreadSafeQueue.h | 45 ++- src/Coordination/tests/gtest_coordination.cpp | 4 +- .../CacheDictionaryUpdateQueue.cpp | 27 +- src/Dictionaries/CacheDictionaryUpdateQueue.h | 4 +- src/Processors/Formats/LazyOutputFormat.cpp | 12 +- src/Processors/Formats/LazyOutputFormat.h | 17 +- .../WriteBufferToRabbitMQProducer.cpp | 3 +- 12 files changed, 236 insertions(+), 179 deletions(-) diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index d50ecda0c9e..921d7bfe37f 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -63,12 +63,10 @@ void ReplicatedAccessStorage::shutdown() bool prev_stop_flag = stop_flag.exchange(true); if (!prev_stop_flag) { + refresh_queue.finish(); + if (worker_thread.joinable()) - { - /// Notify the worker thread to stop waiting for new queue items - refresh_queue.push(UUIDHelpers::Nil); worker_thread.join(); - } } } diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 240c433f923..61b9ad3c1f5 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -3,153 +3,194 @@ #include #include #include - -#include -#include +#include +#include #include -#include -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} -} /** A very simple thread-safe queue of limited size. - * If you try to pop an item from an empty queue, the thread is blocked until the queue becomes nonempty. - * If you try to push an element into an overflowed queue, the thread is blocked until space appears in the queue. + * If you try to pop an item from an empty queue, the thread is blocked until the queue becomes nonempty or queue is finished. + * If you try to push an element into an overflowed queue, the thread is blocked until space appears in the queue or queue is finished. */ template class ConcurrentBoundedQueue { private: std::queue queue; - mutable Poco::FastMutex mutex; - Poco::Semaphore fill_count; - Poco::Semaphore empty_count; - std::atomic_bool closed = false; - template - bool tryEmplaceImpl(Args &&... args) + mutable std::mutex queue_mutex; + std::condition_variable push_condition; + std::condition_variable pop_condition; + + bool is_finished = false; + + size_t max_fill = 0; + + template + bool emplaceImpl(bool wait_on_timeout, UInt64 timeout_milliseconds = 0, Args &&...args) { - bool emplaced = true; - { - Poco::ScopedLock lock(mutex); - if (closed) - emplaced = false; + std::unique_lock queue_lock(queue_mutex); + + auto predicate = [&]() { return is_finished || queue.size() < max_fill; }; + + if (wait_on_timeout) + { + bool wait_result = push_condition.wait_for(queue_lock, std::chrono::milliseconds(timeout_milliseconds), predicate); + + if (!wait_result) + return false; + } else - queue.emplace(std::forward(args)...); + { + push_condition.wait(queue_lock, [&](){ return is_finished || queue.size() < max_fill; }); + } + + if (is_finished) + return false; + + queue.emplace(std::forward(args)...); } - if (emplaced) - fill_count.set(); - else - empty_count.set(); - - return emplaced; - } - - void popImpl(T & x) - { - { - Poco::ScopedLock lock(mutex); - detail::moveOrCopyIfThrow(std::move(queue.front()), x); - queue.pop(); - } - empty_count.set(); - } - -public: - explicit ConcurrentBoundedQueue(size_t max_fill) - : fill_count(0, max_fill) - , empty_count(max_fill, max_fill) - {} - - void push(const T & x) - { - empty_count.wait(); - if (!tryEmplaceImpl(x)) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "tryPush/tryEmplace must be used with close()"); - } - - template - void emplace(Args &&... args) - { - empty_count.wait(); - if (!tryEmplaceImpl(std::forward(args)...)) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "tryPush/tryEmplace must be used with close()"); - } - - void pop(T & x) - { - fill_count.wait(); - popImpl(x); - } - - bool tryPush(const T & x, UInt64 milliseconds = 0) - { - if (!empty_count.tryWait(milliseconds)) - return false; - - return tryEmplaceImpl(x); - } - - template - bool tryEmplace(UInt64 milliseconds, Args &&... args) - { - if (!empty_count.tryWait(milliseconds)) - return false; - - return tryEmplaceImpl(std::forward(args)...); - } - - bool tryPop(T & x, UInt64 milliseconds = 0) - { - if (!fill_count.tryWait(milliseconds)) - return false; - - popImpl(x); + pop_condition.notify_one(); return true; } + bool popImpl(T & x, bool wait_on_timeout, UInt64 timeout_milliseconds = 0) + { + { + std::unique_lock queue_lock(queue_mutex); + + auto predicate = [&]() { return is_finished || !queue.empty(); }; + + if (wait_on_timeout) + { + bool wait_result = pop_condition.wait_for(queue_lock, std::chrono::milliseconds(timeout_milliseconds), predicate); + + if (!wait_result) + return false; + } + else + { + pop_condition.wait(queue_lock, [&](){ return is_finished || !queue.empty(); }); + } + + if (is_finished) + return false; + + detail::moveOrCopyIfThrow(std::move(queue.front()), x); + queue.pop(); + } + + push_condition.notify_one(); + return true; + } + +public: + + explicit ConcurrentBoundedQueue(size_t max_fill_) + : max_fill(max_fill_) + {} + + /// Returns false if queue is finished + bool push(const T & x) + { + return emplace(x); + } + + /// Returns false if queue is finished + template + bool emplace(Args &&... args) + { + emplaceImpl(false /*wait on timeout*/, 0 /* timeout in milliseconds */, std::forward(args...)); + return true; + } + + /// Returns false if queue is finished + [[nodiscard]] bool pop(T & x) + { + return popImpl(x, false /* wait on timeout*/); + } + + /// Returns false if queue is finished or object was not pushed during timeout + bool tryPush(const T & x, UInt64 milliseconds = 0) + { + return emplaceImpl(true /*wait on timeout*/, milliseconds, x); + } + + /// Returns false if queue is finished or object was not emplaced during timeout + template + bool tryEmplace(UInt64 milliseconds, Args &&... args) + { + return emplaceImpl(true /*wait on timeout*/, milliseconds, std::forward(args...)); + } + + /// Returns false if queue is finished or object was not popped during timeout + [[nodiscard]] bool tryPop(T & x, UInt64 milliseconds = 0) + { + return popImpl(x, true /*wait on timeout*/, milliseconds); + } + + /// Returns size of queue size_t size() const { - Poco::ScopedLock lock(mutex); + std::lock_guard lock(queue_mutex); return queue.size(); } - size_t empty() const + /// Returns if queue is empty + bool empty() const { - Poco::ScopedLock lock(mutex); + std::lock_guard lock(queue_mutex); return queue.empty(); } - /// Forbids to push new elements to queue. - /// Returns false if queue was not closed before call, returns true if queue was already closed. - bool close() + /** Clear and finish queue + * After that push or pop operations will return false + * Returns true if queue was already finished + */ + bool finish() { - Poco::ScopedLock lock(mutex); - return closed.exchange(true); + bool was_finished_before = false; + + { + std::lock_guard lock(queue_mutex); + + if (is_finished) + return true; + + std::queue empty_queue; + queue.swap(empty_queue); + + was_finished_before = is_finished; + is_finished = true; + } + + pop_condition.notify_all(); + push_condition.notify_all(); + + return was_finished_before; } - bool isClosed() const + /// Returns if queue is finished + bool isFinished() const { - return closed.load(); + std::lock_guard lock(queue_mutex); + return is_finished; } + /// Clear queue void clear() { - while (fill_count.tryWait(0)) - { - { - Poco::ScopedLock lock(mutex); - queue.pop(); - } - empty_count.set(); - } + std::lock_guard lock(queue_mutex); + + if (is_finished) + return; + + std::queue empty_queue; + queue.swap(empty_queue); + + push_condition.notify_all(); } }; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 59ed906db15..aad423fcbd4 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -545,7 +545,7 @@ void ZooKeeper::sendThread() try { - while (!requests_queue.isClosed()) + while (!requests_queue.isFinished()) { auto prev_bytes_sent = out->count(); @@ -577,7 +577,7 @@ void ZooKeeper::sendThread() info.request->has_watch = true; } - if (requests_queue.isClosed()) + if (requests_queue.isFinished()) { break; } @@ -622,7 +622,7 @@ void ZooKeeper::receiveThread() try { Int64 waited = 0; - while (!requests_queue.isClosed()) + while (!requests_queue.isFinished()) { auto prev_bytes_received = in->count(); @@ -645,7 +645,7 @@ void ZooKeeper::receiveThread() if (in->poll(max_wait)) { - if (requests_queue.isClosed()) + if (requests_queue.isFinished()) break; receiveEvent(); @@ -850,9 +850,9 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea auto expire_session_if_not_expired = [&] { - /// No new requests will appear in queue after close() - bool was_already_closed = requests_queue.close(); - if (!was_already_closed) + /// No new requests will appear in queue after finish() + bool was_already_finished = requests_queue.finish(); + if (!was_already_finished) active_session_metric_increment.destroy(); }; @@ -1026,13 +1026,11 @@ void ZooKeeper::pushRequest(RequestInfo && info) } } - if (requests_queue.isClosed()) - throw Exception("Session expired", Error::ZSESSIONEXPIRED); - if (!requests_queue.tryPush(std::move(info), operation_timeout.totalMilliseconds())) { - if (requests_queue.isClosed()) + if (requests_queue.isFinished()) throw Exception("Session expired", Error::ZSESSIONEXPIRED); + throw Exception("Cannot push request to queue within operation timeout", Error::ZOPERATIONTIMEOUT); } } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 74c0148e7b6..b87469bd339 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -121,7 +121,7 @@ public: /// If expired, you can only destroy the object. All other methods will throw exception. - bool isExpired() const override { return requests_queue.isClosed(); } + bool isExpired() const override { return requests_queue.isFinished(); } /// Useful to check owner of ephemeral node. int64_t getSessionID() const override { return session_id; } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b4dc367ff62..d80c3567853 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -164,7 +164,8 @@ void KeeperDispatcher::snapshotThread() while (!shutdown_called) { CreateSnapshotTask task; - snapshots_queue.pop(task); + if (!snapshots_queue.pop(task)) + break; if (shutdown_called) break; @@ -295,16 +296,15 @@ void KeeperDispatcher::shutdown() if (requests_queue) { - requests_queue->push({}); - if (request_thread.joinable()) - request_thread.join(); + requests_queue->finish(); + request_thread.join(); } - responses_queue.push({}); + responses_queue.finish(); if (responses_thread.joinable()) responses_thread.join(); - snapshots_queue.push({}); + snapshots_queue.finish(); if (snapshot_thread.joinable()) snapshot_thread.join(); } diff --git a/src/Coordination/ThreadSafeQueue.h b/src/Coordination/ThreadSafeQueue.h index d36e25244bb..cb8c3499f73 100644 --- a/src/Coordination/ThreadSafeQueue.h +++ b/src/Coordination/ThreadSafeQueue.h @@ -3,6 +3,9 @@ #include #include +#include + + namespace DB { @@ -14,24 +17,37 @@ private: mutable std::mutex queue_mutex; std::condition_variable cv; std::queue queue; + bool is_finished; public: - void push(const T & response) + bool push(const T & response) { - std::lock_guard lock(queue_mutex); - queue.push(response); + { + std::lock_guard lock(queue_mutex); + + if (is_finished) + return false; + + queue.push(response); + } + cv.notify_one(); + return true; } - bool tryPop(T & response, int64_t timeout_ms = 0) + [[nodiscard]] bool tryPop(T & response, int64_t timeout_ms = 0) { std::unique_lock lock(queue_mutex); if (!cv.wait_for(lock, - std::chrono::milliseconds(timeout_ms), [this] { return !queue.empty(); })) + std::chrono::milliseconds(timeout_ms), [this] { return is_finished || !queue.empty(); })) return false; - response = queue.front(); + if (is_finished) + return false; + + ::detail::moveOrCopyIfThrow(std::move(queue.front()), response); queue.pop(); + return true; } @@ -40,6 +56,23 @@ public: std::lock_guard lock(queue_mutex); return queue.size(); } + + bool isFinished() const + { + std::lock_guard lock(queue_mutex); + return is_finished; + } + + bool finish() + { + std::lock_guard lock(queue_mutex); + bool was_finished_before = is_finished; + is_finished = true; + + cv.notify_all(); + + return was_finished_before; + } }; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 99dd64a21cd..3c85f8c410c 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1186,7 +1186,9 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint state_machine->create_snapshot(s, when_done); CreateSnapshotTask snapshot_task; - snapshots_queue.pop(snapshot_task); + bool pop_result = snapshots_queue.pop(snapshot_task); + EXPECT_TRUE(pop_result); + snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); } if (snapshot_created) diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp index 1d96fcc108b..ba8c0ddffc6 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp @@ -35,9 +35,11 @@ CacheDictionaryUpdateQueue::CacheDictionaryUpdateQueue( template CacheDictionaryUpdateQueue::~CacheDictionaryUpdateQueue() { + if (update_queue.isFinished()) + return; + try { - if (!finished) - stopAndWait(); + stopAndWait(); } catch (...) { @@ -48,7 +50,7 @@ CacheDictionaryUpdateQueue::~CacheDictionaryUpdateQueue() template void CacheDictionaryUpdateQueue::tryPushToUpdateQueueOrThrow(CacheDictionaryUpdateUnitPtr & update_unit_ptr) { - if (finished) + if (update_queue.isFinished()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished"); if (!update_queue.tryPush(update_unit_ptr, configuration.update_queue_push_timeout_milliseconds)) @@ -63,7 +65,7 @@ void CacheDictionaryUpdateQueue::tryPushToUpdateQueueOrThro template void CacheDictionaryUpdateQueue::waitForCurrentUpdateFinish(CacheDictionaryUpdateUnitPtr & update_unit_ptr) const { - if (finished) + if (update_queue.isFinished()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished"); std::unique_lock update_lock(update_mutex); @@ -108,15 +110,10 @@ void CacheDictionaryUpdateQueue::waitForCurrentUpdateFinish template void CacheDictionaryUpdateQueue::stopAndWait() { - finished = true; - update_queue.clear(); - - for (size_t i = 0; i < configuration.max_threads_for_updates; ++i) - { - auto empty_finishing_ptr = std::make_shared>(); - update_queue.push(empty_finishing_ptr); - } + if (update_queue.isFinished()) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished"); + update_queue.finish(); update_pool.wait(); } @@ -125,12 +122,10 @@ void CacheDictionaryUpdateQueue::updateThreadFunction() { setThreadName("UpdQueue"); - while (!finished) + while (!update_queue.isFinished()) { CacheDictionaryUpdateUnitPtr unit_to_update; - update_queue.pop(unit_to_update); - - if (finished) + if (!update_queue.pop(unit_to_update)) break; try diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.h b/src/Dictionaries/CacheDictionaryUpdateQueue.h index bcad376bc53..7725ce7588f 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.h +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.h @@ -122,7 +122,7 @@ public: const CacheDictionaryUpdateQueueConfiguration & getConfiguration() const { return configuration; } /// Is queue finished - bool isFinished() const { return finished; } + bool isFinished() const { return update_queue.isFinished(); } /// Synchronous wait for update queue to stop void stopAndWait(); @@ -162,8 +162,6 @@ private: mutable std::mutex update_mutex; mutable std::condition_variable is_update_finished; - - std::atomic finished{false}; }; extern template class CacheDictionaryUpdateQueue; diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 0663ff28f84..4d919833df2 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -9,11 +9,8 @@ WriteBuffer LazyOutputFormat::out(nullptr, 0); Chunk LazyOutputFormat::getChunk(UInt64 milliseconds) { - if (finished_processing) - { - if (queue.empty()) - return {}; - } + if (queue.isFinished()) + return {}; Chunk chunk; if (milliseconds) @@ -22,7 +19,10 @@ Chunk LazyOutputFormat::getChunk(UInt64 milliseconds) return {}; } else - queue.pop(chunk); + { + if (!queue.pop(chunk)) + return {}; + } if (chunk) info.update(chunk.getNumRows(), chunk.allocatedBytes()); diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 2c29f55c4f3..66749c63f03 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -15,7 +15,7 @@ class LazyOutputFormat : public IOutputFormat public: explicit LazyOutputFormat(const Block & header) - : IOutputFormat(header, out), queue(2), finished_processing(false) {} + : IOutputFormat(header, out), queue(2) {} String getName() const override { return "LazyOutputFormat"; } @@ -23,7 +23,7 @@ public: Chunk getTotals(); Chunk getExtremes(); - bool isFinished() { return finished_processing && queue.size() == 0; } + bool isFinished() { return queue.isFinished(); } BlockStreamProfileInfo & getProfileInfo() { return info; } @@ -31,17 +31,12 @@ public: void onCancel() override { - finished_processing = true; - /// Clear queue in case if somebody is waiting lazy_format to push. - queue.clear(); + queue.finish(); } void finalize() override { - finished_processing = true; - - /// In case we are waiting for result. - queue.emplace(Chunk()); + queue.finish(); } bool expectMaterializedColumns() const override { return false; } @@ -49,7 +44,7 @@ public: protected: void consume(Chunk chunk) override { - if (!finished_processing) + if (!queue.isFinished()) queue.emplace(std::move(chunk)); } @@ -66,8 +61,6 @@ private: static WriteBuffer out; BlockStreamProfileInfo info; - - std::atomic finished_processing; }; } diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 1929a103414..2859cf226ea 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -192,9 +192,8 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable() && delivery_record.size() < RETURNED_LIMIT) + while (messages.pop(payload) && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) { - messages.pop(payload); AMQP::Envelope envelope(payload.second.data(), payload.second.size()); /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty From afdc2fe50d089c70c46cfd9992ee21e41d805a1a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 7 Oct 2021 11:21:01 +0300 Subject: [PATCH 610/950] Fixed tests --- src/Common/ConcurrentBoundedQueue.h | 25 +++++++++++++++++-- src/Coordination/ThreadSafeQueue.h | 2 +- .../CacheDictionaryUpdateQueue.cpp | 2 +- 3 files changed, 25 insertions(+), 4 deletions(-) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 61b9ad3c1f5..8f40d26ce44 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -76,11 +76,13 @@ private: pop_condition.wait(queue_lock, [&](){ return is_finished || !queue.empty(); }); } - if (is_finished) + if (is_finished && queue.empty()) return false; detail::moveOrCopyIfThrow(std::move(queue.front()), x); queue.pop(); + + return true; } push_condition.notify_one(); @@ -147,7 +149,8 @@ public: } /** Clear and finish queue - * After that push or pop operations will return false + * After that push operation will return false + * pop operations will return values until queue become empty * Returns true if queue was already finished */ bool finish() @@ -193,4 +196,22 @@ public: push_condition.notify_all(); } + + /// Clear and finish queue + void clearAndFinish() + { + { + std::lock_guard lock(queue_mutex); + + if (is_finished) + return; + + std::queue empty_queue; + queue.swap(empty_queue); + is_finished = true; + } + + pop_condition.notify_all(); + push_condition.notify_all(); + } }; diff --git a/src/Coordination/ThreadSafeQueue.h b/src/Coordination/ThreadSafeQueue.h index cb8c3499f73..ba79a99f94e 100644 --- a/src/Coordination/ThreadSafeQueue.h +++ b/src/Coordination/ThreadSafeQueue.h @@ -42,7 +42,7 @@ public: std::chrono::milliseconds(timeout_ms), [this] { return is_finished || !queue.empty(); })) return false; - if (is_finished) + if (is_finished && queue.empty()) return false; ::detail::moveOrCopyIfThrow(std::move(queue.front()), response); diff --git a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp index ba8c0ddffc6..2077f846f09 100644 --- a/src/Dictionaries/CacheDictionaryUpdateQueue.cpp +++ b/src/Dictionaries/CacheDictionaryUpdateQueue.cpp @@ -113,7 +113,7 @@ void CacheDictionaryUpdateQueue::stopAndWait() if (update_queue.isFinished()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "CacheDictionaryUpdateQueue finished"); - update_queue.finish(); + update_queue.clearAndFinish(); update_pool.wait(); } From aae409b321f4b5568b0b7d28283dd447ac5fa019 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 7 Oct 2021 20:13:56 +0300 Subject: [PATCH 611/950] Fixed tests --- src/Common/ConcurrentBoundedQueue.h | 49 ++++++++++++--------------- src/Coordination/KeeperDispatcher.cpp | 13 ++----- 2 files changed, 25 insertions(+), 37 deletions(-) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 8f40d26ce44..9a722c5654b 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -28,16 +29,16 @@ private: size_t max_fill = 0; template - bool emplaceImpl(bool wait_on_timeout, UInt64 timeout_milliseconds = 0, Args &&...args) + bool emplaceImpl(std::optional timeout_milliseconds, Args &&...args) { { std::unique_lock queue_lock(queue_mutex); auto predicate = [&]() { return is_finished || queue.size() < max_fill; }; - if (wait_on_timeout) + if (timeout_milliseconds.has_value()) { - bool wait_result = push_condition.wait_for(queue_lock, std::chrono::milliseconds(timeout_milliseconds), predicate); + bool wait_result = push_condition.wait_for(queue_lock, std::chrono::milliseconds(timeout_milliseconds.value()), predicate); if (!wait_result) return false; @@ -57,23 +58,23 @@ private: return true; } - bool popImpl(T & x, bool wait_on_timeout, UInt64 timeout_milliseconds = 0) + bool popImpl(T & x, std::optional timeout_milliseconds) { { std::unique_lock queue_lock(queue_mutex); auto predicate = [&]() { return is_finished || !queue.empty(); }; - if (wait_on_timeout) + if (timeout_milliseconds.has_value()) { - bool wait_result = pop_condition.wait_for(queue_lock, std::chrono::milliseconds(timeout_milliseconds), predicate); + bool wait_result = pop_condition.wait_for(queue_lock, std::chrono::milliseconds(timeout_milliseconds.value()), predicate); if (!wait_result) return false; } else { - pop_condition.wait(queue_lock, [&](){ return is_finished || !queue.empty(); }); + pop_condition.wait(queue_lock, predicate); } if (is_finished && queue.empty()) @@ -81,8 +82,6 @@ private: detail::moveOrCopyIfThrow(std::move(queue.front()), x); queue.pop(); - - return true; } push_condition.notify_one(); @@ -105,33 +104,33 @@ public: template bool emplace(Args &&... args) { - emplaceImpl(false /*wait on timeout*/, 0 /* timeout in milliseconds */, std::forward(args...)); + emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); return true; } - /// Returns false if queue is finished + /// Returns false if queue is finished and empty [[nodiscard]] bool pop(T & x) { - return popImpl(x, false /* wait on timeout*/); + return popImpl(x, std::nullopt /*timeout in milliseconds*/); } /// Returns false if queue is finished or object was not pushed during timeout bool tryPush(const T & x, UInt64 milliseconds = 0) { - return emplaceImpl(true /*wait on timeout*/, milliseconds, x); + return emplaceImpl(milliseconds, x); } /// Returns false if queue is finished or object was not emplaced during timeout template bool tryEmplace(UInt64 milliseconds, Args &&... args) { - return emplaceImpl(true /*wait on timeout*/, milliseconds, std::forward(args...)); + return emplaceImpl(milliseconds, std::forward(args...)); } - /// Returns false if queue is finished or object was not popped during timeout + /// Returns false if queue is (finished and empty) or (object was not popped during timeout) [[nodiscard]] bool tryPop(T & x, UInt64 milliseconds = 0) { - return popImpl(x, true /*wait on timeout*/, milliseconds); + return popImpl(x, milliseconds); } /// Returns size of queue @@ -163,9 +162,6 @@ public: if (is_finished) return true; - std::queue empty_queue; - queue.swap(empty_queue); - was_finished_before = is_finished; is_finished = true; } @@ -186,13 +182,15 @@ public: /// Clear queue void clear() { - std::lock_guard lock(queue_mutex); + { + std::lock_guard lock(queue_mutex); - if (is_finished) - return; + if (is_finished) + return; - std::queue empty_queue; - queue.swap(empty_queue); + std::queue empty_queue; + queue.swap(empty_queue); + } push_condition.notify_all(); } @@ -203,9 +201,6 @@ public: { std::lock_guard lock(queue_mutex); - if (is_finished) - return; - std::queue empty_queue; queue.swap(empty_queue); is_finished = true; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index d80c3567853..e7e5452157c 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -317,16 +317,9 @@ void KeeperDispatcher::shutdown() /// Set session expired for all pending requests while (requests_queue && requests_queue->tryPop(request_for_session)) { - if (request_for_session.request) - { - auto response = request_for_session.request->makeResponse(); - response->error = Coordination::Error::ZSESSIONEXPIRED; - setResponse(request_for_session.session_id, response); - } - else - { - break; - } + auto response = request_for_session.request->makeResponse(); + response->error = Coordination::Error::ZSESSIONEXPIRED; + setResponse(request_for_session.session_id, response); } /// Clear all registered sessions From d55561422f5755d35894f9a8e6801625ef5d5b5e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 8 Oct 2021 01:06:33 +0300 Subject: [PATCH 612/950] Fixed tests --- base/loggers/OwnSplitChannel.cpp | 2 +- src/Common/ConcurrentBoundedQueue.h | 8 ++++---- src/Coordination/KeeperDispatcher.cpp | 11 +++++++++-- src/Coordination/KeeperStateMachine.cpp | 3 ++- src/Interpreters/InternalTextLogsQueue.cpp | 2 +- src/Processors/Formats/LazyOutputFormat.h | 2 +- .../RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp | 5 +++-- .../RabbitMQ/WriteBufferToRabbitMQProducer.cpp | 12 ++++++++---- 8 files changed, 29 insertions(+), 16 deletions(-) diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 62fdddad28c..16c9fedf21d 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -100,7 +100,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) columns[i++]->insert(msg.getSource()); columns[i++]->insert(msg.getText()); - logs_queue->emplace(std::move(columns)); + (void)(logs_queue->emplace(std::move(columns))); } /// Also log to system.text_log table, if message is not too noisy diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index 9a722c5654b..fdd1c7cc388 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -95,14 +95,14 @@ public: {} /// Returns false if queue is finished - bool push(const T & x) + [[nodiscard]] bool push(const T & x) { return emplace(x); } /// Returns false if queue is finished template - bool emplace(Args &&... args) + [[nodiscard]] bool emplace(Args &&... args) { emplaceImpl(std::nullopt /* timeout in milliseconds */, std::forward(args...)); return true; @@ -115,14 +115,14 @@ public: } /// Returns false if queue is finished or object was not pushed during timeout - bool tryPush(const T & x, UInt64 milliseconds = 0) + [[nodiscard]] bool tryPush(const T & x, UInt64 milliseconds = 0) { return emplaceImpl(milliseconds, x); } /// Returns false if queue is finished or object was not emplaced during timeout template - bool tryEmplace(UInt64 milliseconds, Args &&... args) + [[nodiscard]] bool tryEmplace(UInt64 milliseconds, Args &&... args) { return emplaceImpl(milliseconds, std::forward(args...)); } diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index e7e5452157c..8f465e59ed5 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -236,9 +236,15 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ /// Put close requests without timeouts if (request->getOpNum() == Coordination::OpNum::Close) - requests_queue->push(std::move(request_info)); + { + if (!requests_queue->push(std::move(request_info))) + throw Exception("Cannot push request to queue", ErrorCodes::LOGICAL_ERROR); + } else if (!requests_queue->tryPush(std::move(request_info), coordination_settings->operation_timeout_ms.totalMilliseconds())) + { throw Exception("Cannot push request to queue within operation timeout", ErrorCodes::TIMEOUT_EXCEEDED); + } + return true; } @@ -372,7 +378,8 @@ void KeeperDispatcher::sessionCleanerTask() request_info.session_id = dead_session; { std::lock_guard lock(push_request_mutex); - requests_queue->push(std::move(request_info)); + if (!requests_queue->push(std::move(request_info))) + LOG_INFO(log, "Cannot push request to queue"); } /// Remove session from registered sessions diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 682a523fcaf..2617c01e24f 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -218,7 +218,8 @@ void KeeperStateMachine::create_snapshot( LOG_DEBUG(log, "In memory snapshot {} created, queueing task to flash to disk", s.get_last_log_idx()); /// Flush snapshot to disk in a separate thread. - snapshots_queue.push(std::move(snapshot_task)); + if (!snapshots_queue.push(std::move(snapshot_task))) + LOG_DEBUG(log, "Cannot push snapshot task into queue"); } void KeeperStateMachine::save_logical_snp_obj( diff --git a/src/Interpreters/InternalTextLogsQueue.cpp b/src/Interpreters/InternalTextLogsQueue.cpp index 8329fc38ba7..e541e1c76cc 100644 --- a/src/Interpreters/InternalTextLogsQueue.cpp +++ b/src/Interpreters/InternalTextLogsQueue.cpp @@ -41,7 +41,7 @@ void InternalTextLogsQueue::pushBlock(Block && log_block) static Block sample_block = getSampleBlock(); if (blocksHaveEqualStructure(sample_block, log_block)) - emplace(log_block.mutateColumns()); + (void)(emplace(log_block.mutateColumns())); else LOG_WARNING(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure"); } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 66749c63f03..91309be8570 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -45,7 +45,7 @@ protected: void consume(Chunk chunk) override { if (!queue.isFinished()) - queue.emplace(std::move(chunk)); + (void)(queue.emplace(std::move(chunk))); } void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 2e2807d8297..6f42aeb9776 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -64,9 +64,10 @@ void ReadBufferFromRabbitMQConsumer::subscribe() if (row_delimiter != '\0') message_received += row_delimiter; - received.push({message_received, message.hasMessageID() ? message.messageID() : "", + if (!received.push({message_received, message.hasMessageID() ? message.messageID() : "", message.hasTimestamp() ? message.timestamp() : 0, - redelivered, AckTracker(delivery_tag, channel_id)}); + redelivered, AckTracker(delivery_tag, channel_id)})) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to received queue"); } }) .onError([&](const char * message) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 2859cf226ea..7a9ec1d567b 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -102,7 +102,8 @@ void WriteBufferToRabbitMQProducer::countRow() reinitializeChunks(); ++payload_counter; - payloads.push(std::make_pair(payload_counter, payload)); + if (!payloads.push(std::make_pair(payload_counter, payload))) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to payloads queue"); } } @@ -122,7 +123,8 @@ void WriteBufferToRabbitMQProducer::setupChannel() * they are republished because after channel recovery they will acquire new delivery tags, so all previous records become invalid */ for (const auto & record : delivery_record) - returned.tryPush(record.second); + if (!returned.push(record.second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); LOG_DEBUG(log, "Producer on channel {} hasn't confirmed {} messages, {} waiting to be published", channel_id, delivery_record.size(), payloads.size()); @@ -170,7 +172,8 @@ void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, b if (republish) for (auto record = delivery_record.begin(); record != record_iter; ++record) - returned.tryPush(record->second); + if (!returned.push(record->second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); /// Delete the records even in case when republished because new delivery tags will be assigned by the server. delivery_record.erase(delivery_record.begin(), record_iter); @@ -178,7 +181,8 @@ void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, b else { if (republish) - returned.tryPush(record_iter->second); + if (returned.push(record_iter->second)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); delivery_record.erase(record_iter); } From 04047f76c75a4cc2826eca016c776c95261071d4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 8 Oct 2021 11:48:08 +0300 Subject: [PATCH 613/950] Fixed tests --- base/loggers/OwnSplitChannel.cpp | 2 +- src/Access/ReplicatedAccessStorage.cpp | 5 +- src/Access/ReplicatedAccessStorage.h | 17 ++-- src/Common/ConcurrentBoundedQueue.h | 7 ++ src/Coordination/KeeperDispatcher.cpp | 8 +- src/Coordination/KeeperStateMachine.cpp | 9 ++- src/Coordination/KeeperStateMachine.h | 5 +- src/Coordination/ThreadSafeQueue.h | 78 ------------------- src/Coordination/tests/gtest_coordination.cpp | 4 +- src/Processors/Formats/LazyOutputFormat.cpp | 2 +- src/Processors/Formats/LazyOutputFormat.h | 7 +- src/Server/KeeperTCPHandler.cpp | 4 +- src/Server/KeeperTCPHandler.h | 4 +- .../ReadBufferFromRabbitMQConsumer.cpp | 5 ++ .../WriteBufferToRabbitMQProducer.cpp | 3 +- utils/keeper-data-dumper/main.cpp | 2 +- 16 files changed, 55 insertions(+), 107 deletions(-) delete mode 100644 src/Coordination/ThreadSafeQueue.h diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 16c9fedf21d..2349c60856f 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -100,7 +100,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) columns[i++]->insert(msg.getSource()); columns[i++]->insert(msg.getText()); - (void)(logs_queue->emplace(std::move(columns))); + [[maybe_unused]] bool push_result = logs_queue->emplace(std::move(columns)); } /// Also log to system.text_log table, if message is not too noisy diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 921d7bfe37f..7b29aab3a89 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -34,6 +34,7 @@ ReplicatedAccessStorage::ReplicatedAccessStorage( : IAccessStorage(storage_name_) , zookeeper_path(zookeeper_path_) , get_zookeeper(get_zookeeper_) + , refresh_queue(std::numeric_limits::max()) { if (zookeeper_path.empty()) throw Exception("ZooKeeper path must be non-empty", ErrorCodes::BAD_ARGUMENTS); @@ -366,7 +367,7 @@ void ReplicatedAccessStorage::refreshEntities(const zkutil::ZooKeeperPtr & zooke const String zookeeper_uuids_path = zookeeper_path + "/uuid"; auto watch_entities_list = [this](const Coordination::WatchResponse &) { - refresh_queue.push(UUIDHelpers::Nil); + [[maybe_unused]] bool push_result = refresh_queue.push(UUIDHelpers::Nil); }; Coordination::Stat stat; const auto entity_uuid_strs = zookeeper->getChildrenWatch(zookeeper_uuids_path, &stat, watch_entities_list); @@ -418,7 +419,7 @@ void ReplicatedAccessStorage::refreshEntityNoLock(const zkutil::ZooKeeperPtr & z const auto watch_entity = [this, id](const Coordination::WatchResponse & response) { if (response.type == Coordination::Event::CHANGED) - refresh_queue.push(id); + [[maybe_unused]] bool push_result = refresh_queue.push(id); }; Coordination::Stat entity_stat; const String entity_path = zookeeper_path + "/uuid/" + toString(id); diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 0df1d5ef5ff..458bc0d614b 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -1,17 +1,20 @@ #pragma once -#include -#include -#include -#include -#include -#include #include #include #include #include #include +#include + +#include +#include +#include +#include + +#include + namespace DB { @@ -36,7 +39,7 @@ private: std::atomic initialized = false; std::atomic stop_flag = false; ThreadFromGlobalPool worker_thread; - ThreadSafeQueue refresh_queue; + ConcurrentBoundedQueue refresh_queue; UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override; void removeImpl(const UUID & id) override; diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index fdd1c7cc388..e15aa462f1c 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -179,6 +179,13 @@ public: return is_finished; } + /// Returns if queue is finished and empty + bool isFinishedAndEmpty() const + { + std::lock_guard lock(queue_mutex); + return is_finished && queue.empty(); + } + /// Clear queue void clear() { diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 8f465e59ed5..3823a4129e3 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes KeeperDispatcher::KeeperDispatcher() : coordination_settings(std::make_shared()) + , responses_queue(std::numeric_limits::max()) , log(&Poco::Logger::get("KeeperDispatcher")) { } @@ -414,7 +415,12 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession response->xid = request->xid; response->zxid = 0; response->error = error; - responses_queue.push(DB::KeeperStorage::ResponseForSession{session_id, response}); + if (!responses_queue.push(DB::KeeperStorage::ResponseForSession{session_id, response})) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Could not push error response xid {} zxid {} error message {} to responses queue", + response->xid, + response->zxid, + errorMessage(error)); } } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 2617c01e24f..b1c8aab697e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -120,7 +120,8 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n session_id = storage->getSessionID(session_id_request.session_timeout_ms); LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; - responses_queue.push(response_for_session); + if (!responses_queue.push(response_for_session)) + LOG_WARNING(log, "Could not push response {} into responses queue", session_id); } } else @@ -128,7 +129,8 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n std::lock_guard lock(storage_and_responses_lock); KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session.request, request_for_session.session_id, log_idx); for (auto & response_for_session : responses_for_sessions) - responses_queue.push(response_for_session); + if (!responses_queue.push(response_for_session)) + LOG_WARNING(log, "Could not push response {} into responses queue", response_for_session.session_id); } last_committed_idx = log_idx; @@ -305,7 +307,8 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi std::lock_guard lock(storage_and_responses_lock); auto responses = storage->processRequest(request_for_session.request, request_for_session.session_id, std::nullopt); for (const auto & response : responses) - responses_queue.push(response); + if (!responses_queue.push(response)) + LOG_WARNING(log, "Could not push response {} into responses queue", response.session_id); } std::vector KeeperStateMachine::getDeadSessions() diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index fcf9c7d14c4..983692f7b7f 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -1,16 +1,17 @@ #pragma once +#include #include #include // Y_IGNORE #include -#include #include #include + namespace DB { -using ResponsesQueue = ThreadSafeQueue; +using ResponsesQueue = ConcurrentBoundedQueue; using SnapshotsQueue = ConcurrentBoundedQueue; /// ClickHouse Keeper state machine. Wrapper for KeeperStorage. diff --git a/src/Coordination/ThreadSafeQueue.h b/src/Coordination/ThreadSafeQueue.h deleted file mode 100644 index ba79a99f94e..00000000000 --- a/src/Coordination/ThreadSafeQueue.h +++ /dev/null @@ -1,78 +0,0 @@ -#pragma once - -#include -#include - -#include - - -namespace DB -{ - -/// Queue with mutex and condvar. As simple as possible. -template -class ThreadSafeQueue -{ -private: - mutable std::mutex queue_mutex; - std::condition_variable cv; - std::queue queue; - bool is_finished; -public: - - bool push(const T & response) - { - { - std::lock_guard lock(queue_mutex); - - if (is_finished) - return false; - - queue.push(response); - } - - cv.notify_one(); - return true; - } - - [[nodiscard]] bool tryPop(T & response, int64_t timeout_ms = 0) - { - std::unique_lock lock(queue_mutex); - if (!cv.wait_for(lock, - std::chrono::milliseconds(timeout_ms), [this] { return is_finished || !queue.empty(); })) - return false; - - if (is_finished && queue.empty()) - return false; - - ::detail::moveOrCopyIfThrow(std::move(queue.front()), response); - queue.pop(); - - return true; - } - - size_t size() const - { - std::lock_guard lock(queue_mutex); - return queue.size(); - } - - bool isFinished() const - { - std::lock_guard lock(queue_mutex); - return is_finished; - } - - bool finish() - { - std::lock_guard lock(queue_mutex); - bool was_finished_before = is_finished; - is_finished = true; - - cv.notify_all(); - - return was_finished_before; - } -}; - -} diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 3c85f8c410c..0cb5972f718 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1159,7 +1159,7 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint ChangelogDirTest snapshots("./snapshots"); ChangelogDirTest logs("./logs"); - ResponsesQueue queue; + ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings); state_machine->init(); @@ -1310,7 +1310,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove) ChangelogDirTest snapshots("./snapshots"); CoordinationSettingsPtr settings = std::make_shared(); - ResponsesQueue queue; + ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings); state_machine->init(); diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 4d919833df2..792d805eac3 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -9,7 +9,7 @@ WriteBuffer LazyOutputFormat::out(nullptr, 0); Chunk LazyOutputFormat::getChunk(UInt64 milliseconds) { - if (queue.isFinished()) + if (isFinished()) return {}; Chunk chunk; diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 91309be8570..50dc87f2e70 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -23,7 +23,7 @@ public: Chunk getTotals(); Chunk getExtremes(); - bool isFinished() { return queue.isFinished(); } + bool isFinished() { return queue.isFinishedAndEmpty(); } BlockStreamProfileInfo & getProfileInfo() { return info; } @@ -31,7 +31,7 @@ public: void onCancel() override { - queue.finish(); + queue.clearAndFinish(); } void finalize() override @@ -44,8 +44,7 @@ public: protected: void consume(Chunk chunk) override { - if (!queue.isFinished()) - (void)(queue.emplace(std::move(chunk))); + (void)(queue.emplace(std::move(chunk))); } void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 7ebbda9dfe6..48519f0af30 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -198,7 +198,7 @@ KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSoc , operation_timeout(0, global_context->getConfigRef().getUInt("keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000) , session_timeout(0, global_context->getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000) , poll_wrapper(std::make_unique(socket_)) - , responses(std::make_unique()) + , responses(std::make_unique(std::numeric_limits::max())) { } @@ -314,7 +314,7 @@ void KeeperTCPHandler::runImpl() auto response_fd = poll_wrapper->getResponseFD(); auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) { - responses->push(response); + [[maybe_unused]] bool push_result = responses->push(response); UInt8 single_byte = 1; [[maybe_unused]] int result = write(response_fd, &single_byte, sizeof(single_byte)); }; diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index 7abfb72c846..274fb21af63 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -13,10 +13,10 @@ #include #include #include +#include #include #include #include -#include #include namespace DB @@ -25,7 +25,7 @@ namespace DB struct SocketInterruptablePollWrapper; using SocketInterruptablePollWrapperPtr = std::unique_ptr; -using ThreadSafeResponseQueue = ThreadSafeQueue; +using ThreadSafeResponseQueue = ConcurrentBoundedQueue; using ThreadSafeResponseQueuePtr = std::unique_ptr; diff --git a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp index 6f42aeb9776..ac60d748e36 100644 --- a/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp +++ b/src/Storages/RabbitMQ/ReadBufferFromRabbitMQConsumer.cpp @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + ReadBufferFromRabbitMQConsumer::ReadBufferFromRabbitMQConsumer( ChannelPtr consumer_channel_, RabbitMQHandler & event_handler_, diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 7a9ec1d567b..c4d5b0e9a23 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -21,6 +21,7 @@ static const auto RETURNED_LIMIT = 50000; namespace ErrorCodes { extern const int CANNOT_CONNECT_RABBITMQ; + extern const int LOGICAL_ERROR; } WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( @@ -181,7 +182,7 @@ void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, b else { if (republish) - if (returned.push(record_iter->second)) + if (!returned.push(record_iter->second)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not push to returned queue"); delivery_record.erase(record_iter); diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index b238c2ef569..ed6a7aea972 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -59,7 +59,7 @@ int main(int argc, char *argv[]) Poco::Logger::root().setLevel("trace"); } auto * logger = &Poco::Logger::get("keeper-dumper"); - ResponsesQueue queue; + ResponsesQueue queue(std::numeric_limits::max()); SnapshotsQueue snapshots_queue{1}; CoordinationSettingsPtr settings = std::make_shared(); auto state_machine = std::make_shared(queue, snapshots_queue, argv[1], settings); From 46ba6498218af0fd5a9bd4bc121ac5ff85a603a4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 8 Oct 2021 16:32:02 +0300 Subject: [PATCH 614/950] Fixed tests --- src/Coordination/KeeperDispatcher.cpp | 11 +++++++---- src/Coordination/KeeperStateMachine.cpp | 7 ++++--- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 3823a4129e3..a28e8d96915 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -11,6 +11,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int TIMEOUT_EXCEEDED; + extern const int SYSTEM_ERROR; } KeeperDispatcher::KeeperDispatcher() @@ -239,7 +240,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ if (request->getOpNum() == Coordination::OpNum::Close) { if (!requests_queue->push(std::move(request_info))) - throw Exception("Cannot push request to queue", ErrorCodes::LOGICAL_ERROR); + throw Exception("Cannot push request to queue", ErrorCodes::SYSTEM_ERROR); } else if (!requests_queue->tryPush(std::move(request_info), coordination_settings->operation_timeout_ms.totalMilliseconds())) { @@ -304,7 +305,9 @@ void KeeperDispatcher::shutdown() if (requests_queue) { requests_queue->finish(); - request_thread.join(); + + if (request_thread.joinable()) + request_thread.join(); } responses_queue.finish(); @@ -380,7 +383,7 @@ void KeeperDispatcher::sessionCleanerTask() { std::lock_guard lock(push_request_mutex); if (!requests_queue->push(std::move(request_info))) - LOG_INFO(log, "Cannot push request to queue"); + LOG_INFO(log, "Cannot push close request to queue while cleaning outdated sessions"); } /// Remove session from registered sessions @@ -416,7 +419,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession response->zxid = 0; response->error = error; if (!responses_queue.push(DB::KeeperStorage::ResponseForSession{session_id, response})) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push error response xid {} zxid {} error message {} to responses queue", response->xid, response->zxid, diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index b1c8aab697e..0a15b504128 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SYSTEM_ERROR; } namespace @@ -130,7 +131,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n KeeperStorage::ResponsesForSessions responses_for_sessions = storage->processRequest(request_for_session.request, request_for_session.session_id, log_idx); for (auto & response_for_session : responses_for_sessions) if (!responses_queue.push(response_for_session)) - LOG_WARNING(log, "Could not push response {} into responses queue", response_for_session.session_id); + throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", response_for_session.session_id); } last_committed_idx = log_idx; @@ -221,7 +222,7 @@ void KeeperStateMachine::create_snapshot( LOG_DEBUG(log, "In memory snapshot {} created, queueing task to flash to disk", s.get_last_log_idx()); /// Flush snapshot to disk in a separate thread. if (!snapshots_queue.push(std::move(snapshot_task))) - LOG_DEBUG(log, "Cannot push snapshot task into queue"); + LOG_WARNING(log, "Cannot push snapshot task into queue"); } void KeeperStateMachine::save_logical_snp_obj( @@ -308,7 +309,7 @@ void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSessi auto responses = storage->processRequest(request_for_session.request, request_for_session.session_id, std::nullopt); for (const auto & response : responses) if (!responses_queue.push(response)) - LOG_WARNING(log, "Could not push response {} into responses queue", response.session_id); + throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", response.session_id); } std::vector KeeperStateMachine::getDeadSessions() From c6eece5a03d4fd6f95a36a1d4ca296496397a7b4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 8 Oct 2021 20:18:33 +0300 Subject: [PATCH 615/950] Fixed tests --- src/Common/ConcurrentBoundedQueue.h | 2 +- src/Coordination/KeeperStateMachine.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ConcurrentBoundedQueue.h b/src/Common/ConcurrentBoundedQueue.h index e15aa462f1c..4e422f7482e 100644 --- a/src/Common/ConcurrentBoundedQueue.h +++ b/src/Common/ConcurrentBoundedQueue.h @@ -45,7 +45,7 @@ private: } else { - push_condition.wait(queue_lock, [&](){ return is_finished || queue.size() < max_fill; }); + push_condition.wait(queue_lock, predicate); } if (is_finished) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 0a15b504128..fe64f8afde7 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -122,7 +122,7 @@ nuraft::ptr KeeperStateMachine::commit(const uint64_t log_idx, n LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms); response->session_id = session_id; if (!responses_queue.push(response_for_session)) - LOG_WARNING(log, "Could not push response {} into responses queue", session_id); + throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push response with session id {} into responses queue", session_id); } } else From 201523f4e6c193e521ae870ce5114c221c95b011 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 11 Oct 2021 19:22:44 +0300 Subject: [PATCH 616/950] Fixed build --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index aad423fcbd4..63b4a61b866 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -842,8 +842,8 @@ void ZooKeeper::finalize(bool error_send, bool error_receive, const String & rea /// If some thread (send/receive) already finalizing session don't try to do it bool already_started = finalization_started.exchange(true); - LOG_TEST(log, "Finalizing session {}: finalization_started={}, queue_closed={}, reason={}", - session_id, already_started, requests_queue.isClosed(), reason); + LOG_TEST(log, "Finalizing session {}: finalization_started={}, queue_finished={}, reason={}", + session_id, already_started, requests_queue.isFinished(), reason); if (already_started) return; From 3d2e1a24d998ba87a0782420200afc429bcb97ce Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 12 Oct 2021 18:41:38 +0300 Subject: [PATCH 617/950] Fixed tests --- src/Server/KeeperTCPHandler.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 48519f0af30..b19b02f960d 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -314,7 +314,12 @@ void KeeperTCPHandler::runImpl() auto response_fd = poll_wrapper->getResponseFD(); auto response_callback = [this, response_fd] (const Coordination::ZooKeeperResponsePtr & response) { - [[maybe_unused]] bool push_result = responses->push(response); + if (!responses->push(response)) + throw Exception(ErrorCodes::SYSTEM_ERROR, + "Could not push response with xid {} and zxid {}", + response->xid, + response->zxid); + UInt8 single_byte = 1; [[maybe_unused]] int result = write(response_fd, &single_byte, sizeof(single_byte)); }; From 35f2d914dc805d8295ab9a7d0d3470dbee44a62d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 00:15:05 +0300 Subject: [PATCH 618/950] Fixed build --- src/DataStreams/RemoteQueryExecutor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 08d3db748b7..b6a5e6f63d0 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -35,6 +35,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int UNKNOWN_PACKET_FROM_SERVER; extern const int DUPLICATED_PART_UUIDS; + extern const int SYSTEM_ERROR; } RemoteQueryExecutor::RemoteQueryExecutor( @@ -396,7 +397,8 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) case Protocol::Server::ProfileEvents: /// Pass profile events from remote server to client if (auto profile_queue = CurrentThread::getInternalProfileEventsQueue()) - profile_queue->emplace(std::move(packet.block)); + if (!profile_queue->emplace(std::move(packet.block))) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Could not push into profile queue"); break; default: From 659a46a9726a45b7bd2aa0819efd2bb600be8c90 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 20:10:41 +0300 Subject: [PATCH 619/950] WriteBufferToRabbitMQProduced fixed queue usage --- src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index c4d5b0e9a23..8d891e34a64 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -197,8 +197,13 @@ void WriteBufferToRabbitMQProducer::publish(ConcurrentBoundedQueueusable() && delivery_record.size() < RETURNED_LIMIT) + while (!messages.empty() && producer_channel->usable() && delivery_record.size() < RETURNED_LIMIT) { + bool pop_result = messages.pop(payload); + + if (!pop_result) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not pop payload"); + AMQP::Envelope envelope(payload.second.data(), payload.second.size()); /// if headers exchange is used, routing keys are added here via headers, if not - it is just empty From db9f72ce0fe5f15d77da1d7809b77ccb59c00fb0 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 14 Oct 2021 00:56:38 +0300 Subject: [PATCH 620/950] Apply suggestions from code review Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 21f63b65181..c347b8ca346 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -360,7 +360,6 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); └──────┴────────────────────────────────────────────────────┘ ``` - ## toDecimal(32\|64\|128\|256)OrDefault {#todecimal3264128256ordefault} Преобразует входную строку в значение с типом данных [Decimal(P,S)](../../sql-reference/data-types/decimal.md). Семейство функций включает в себя: @@ -854,6 +853,7 @@ SELECT ## accurateCastOrDefault(x, T[, default_value]) {#type_conversion_function-accurate-cast_or_default} + Преобразует входное значение `x` в указанный тип данных `T`. Если исходное значение не может быть преобразовано к целевому типу, возвращает значение по умолчанию или `default_value`, если оно указано. **Синтаксис** From ef1a01603f917ab897a3df6f0ffbe5635587dc55 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Thu, 14 Oct 2021 01:17:34 +0300 Subject: [PATCH 621/950] Update type-conversion-functions.md --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 21f63b65181..8e010552772 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -866,7 +866,7 @@ accurateCastOrDefault(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. -- `default_value` — значение по умолчанию возвращаемого типа данных. +- `default_value` - значение по умолчанию возвращаемого типа данных. **Возвращаемое значение** From 7c6ef01a04310d175b250d1af8b22abb13d9fbd9 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Thu, 14 Oct 2021 02:00:40 +0300 Subject: [PATCH 622/950] em dash fixed --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 2abb78b1bb7..c36861f280a 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -869,7 +869,7 @@ accurateCastOrDefault(x, T) - `x` — Input value. - `T` — The name of the returned data type. -- `default_value` - Default value of returned data type. +- `default_value` — Default value of returned data type. **Returned value** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 0d5f1c11a7b..c347b8ca346 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -866,7 +866,7 @@ accurateCastOrDefault(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. -- `default_value` - значение по умолчанию возвращаемого типа данных. +- `default_value` — значение по умолчанию возвращаемого типа данных. **Возвращаемое значение** From 3116767834dd59cd6a8ed444966911bef060f12e Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 14 Oct 2021 02:09:43 +0000 Subject: [PATCH 623/950] fix --- src/Storages/FileLog/DirectoryWatcherBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index c9530453e2e..db89746358b 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -57,7 +57,7 @@ public: DirectoryEvent(const std::string & f, DirectoryEventType ev) : path(f), event(ev) { } /// The directory or file that has been changed. - const std::string & path; + const std::string path; /// The kind of event. DirectoryEventType event; }; From 92413aed68a0bbe01b712354949c4bf88f30b698 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Oct 2021 01:01:00 +0300 Subject: [PATCH 624/950] better interfaces for IDataType and ISerialization --- src/Compression/CompressionFactory.h | 8 +- .../CompressionFactoryAdditions.cpp | 13 +- src/Core/Block.cpp | 2 +- src/DataTypes/DataTypeAggregateFunction.cpp | 2 - src/DataTypes/DataTypeArray.cpp | 71 ----------- src/DataTypes/DataTypeArray.h | 11 -- src/DataTypes/DataTypeDate.cpp | 7 -- src/DataTypes/DataTypeDateTime.cpp | 18 +-- src/DataTypes/DataTypeDateTime.h | 24 +--- src/DataTypes/DataTypeDateTime64.cpp | 14 +-- src/DataTypes/DataTypeDecimalBase.cpp | 10 -- src/DataTypes/DataTypeEnum.cpp | 1 - src/DataTypes/DataTypeFixedString.cpp | 10 -- src/DataTypes/DataTypeMap.cpp | 30 ----- src/DataTypes/DataTypeMap.h | 5 - src/DataTypes/DataTypeNested.cpp | 1 - src/DataTypes/DataTypeNothing.cpp | 3 - src/DataTypes/DataTypeNullable.cpp | 34 ----- src/DataTypes/DataTypeNullable.h | 5 - src/DataTypes/DataTypeNumberBase.cpp | 7 -- src/DataTypes/DataTypeString.cpp | 17 --- src/DataTypes/DataTypeString.h | 2 - src/DataTypes/DataTypeTuple.cpp | 82 +----------- src/DataTypes/DataTypeTuple.h | 12 +- src/DataTypes/DataTypesDecimal.cpp | 3 - src/DataTypes/IDataType.cpp | 112 ++++++++--------- src/DataTypes/IDataType.h | 35 +++--- .../Serializations/ISerialization.cpp | 119 ++++++++++++------ src/DataTypes/Serializations/ISerialization.h | 56 +++++++-- .../SerializationAggregateFunction.cpp | 1 - .../Serializations/SerializationArray.cpp | 54 ++++++-- .../Serializations/SerializationArray.h | 18 ++- .../Serializations/SerializationDate32.cpp | 1 + .../Serializations/SerializationDateTime.cpp | 5 +- .../Serializations/SerializationDateTime.h | 9 +- .../SerializationDateTime64.cpp | 4 +- .../Serializations/SerializationDateTime64.h | 9 +- .../SerializationLowCardinality.cpp | 20 ++- .../SerializationLowCardinality.h | 6 +- .../Serializations/SerializationMap.cpp | 13 +- .../Serializations/SerializationMap.h | 6 +- ...upleElement.cpp => SerializationNamed.cpp} | 23 ++-- ...ionTupleElement.h => SerializationNamed.h} | 26 +++- .../Serializations/SerializationNullable.cpp | 50 +++++++- .../Serializations/SerializationNullable.h | 18 ++- .../Serializations/SerializationTuple.cpp | 19 ++- .../Serializations/SerializationTuple.h | 10 +- .../Serializations/SerializationWrapper.cpp | 8 +- .../Serializations/SerializationWrapper.h | 6 +- src/DataTypes/TimezoneMixin.h | 32 +++++ src/Interpreters/InterpreterDescribeQuery.cpp | 6 +- .../MergeTree/IMergedBlockOutputStream.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 9 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 10 +- .../MergeTree/MergeTreeReaderCompact.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 5 +- src/Storages/MergeTree/checkDataPart.cpp | 5 +- src/Storages/StorageLog.cpp | 12 +- src/Storages/StorageTinyLog.cpp | 6 +- 60 files changed, 526 insertions(+), 586 deletions(-) rename src/DataTypes/Serializations/{SerializationTupleElement.cpp => SerializationNamed.cpp} (69%) rename src/DataTypes/Serializations/{SerializationTupleElement.h => SerializationNamed.h} (59%) create mode 100644 src/DataTypes/TimezoneMixin.h diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index f00e5071990..2101dc28c65 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -40,13 +40,7 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const; - - /// Just wrapper for previous method. - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const - { - return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check, allow_experimental_codecs); - } + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const; /// Validate codecs AST specified by user void validateCodec(const String & family_name, std::optional level, bool sanity_check, bool allow_experimental_codecs) const; diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index 9e0353e6711..b5f00c60827 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -53,7 +53,7 @@ void CompressionCodecFactory::validateCodec( ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( - const ASTPtr & ast, const IDataType * column_type, bool sanity_check, bool allow_experimental_codecs) const + const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check, bool allow_experimental_codecs) const { if (const auto * func = ast->as()) { @@ -100,12 +100,13 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( if (column_type) { CompressionCodecPtr prev_codec; - IDataType::StreamCallbackWithType callback = [&]( - const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) + ISerialization::StreamCallback callback = [&](const auto & substream_path) { + assert(!substream_path.empty()); if (ISerialization::isSpecialCompressionAllowed(substream_path)) { - result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); + const auto & last_type = substream_path.back().data.type; + result_codec = getImpl(codec_family_name, codec_arguments, last_type.get()); /// Case for column Tuple, which compressed with codec which depends on data type, like Delta. /// We cannot substitute parameters for such codecs. @@ -115,8 +116,8 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( } }; - ISerialization::SubstreamPath stream_path; - column_type->enumerateStreams(column_type->getDefaultSerialization(), callback, stream_path); + ISerialization::SubstreamPath path; + column_type->getDefaultSerialization()->enumerateStreams(path, callback, column_type, nullptr); if (!result_codec) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a59ac60155e..5f9cb19374b 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -702,7 +702,7 @@ ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & column current_column = current_column->decompress(); if (column.isSubcolumn()) - return column.getTypeInStorage()->getSubcolumn(column.getSubcolumnName(), *current_column); + return column.getTypeInStorage()->getSubcolumn(column.getSubcolumnName(), current_column); return current_column; } diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index f7ae3170119..5c4b94ad823 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -3,8 +3,6 @@ #include -#include -#include #include #include diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index bcf3a9c1f57..f78aebd2d99 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -1,17 +1,9 @@ #include -#include -#include -#include -#include - #include -#include #include #include #include -#include -#include #include @@ -53,69 +45,6 @@ bool DataTypeArray::equals(const IDataType & rhs) const return typeid(rhs) == typeid(*this) && nested->equals(*static_cast(rhs).nested); } -DataTypePtr DataTypeArray::tryGetSubcolumnType(const String & subcolumn_name) const -{ - return tryGetSubcolumnTypeImpl(subcolumn_name, 0); -} - -DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const -{ - if (subcolumn_name == "size" + std::to_string(level)) - return std::make_shared(); - - DataTypePtr subcolumn; - if (const auto * nested_array = typeid_cast(nested.get())) - subcolumn = nested_array->tryGetSubcolumnTypeImpl(subcolumn_name, level + 1); - else - subcolumn = nested->tryGetSubcolumnType(subcolumn_name); - - if (subcolumn && subcolumn_name != MAIN_SUBCOLUMN_NAME) - subcolumn = std::make_shared(std::move(subcolumn)); - - return subcolumn; -} - -ColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, const IColumn & column) const -{ - return getSubcolumnImpl(subcolumn_name, column, 0); -} - -ColumnPtr DataTypeArray::getSubcolumnImpl(const String & subcolumn_name, const IColumn & column, size_t level) const -{ - const auto & column_array = assert_cast(column); - if (subcolumn_name == "size" + std::to_string(level)) - return arrayOffsetsToSizes(column_array.getOffsetsColumn()); - - ColumnPtr subcolumn; - if (const auto * nested_array = typeid_cast(nested.get())) - subcolumn = nested_array->getSubcolumnImpl(subcolumn_name, column_array.getData(), level + 1); - else - subcolumn = nested->getSubcolumn(subcolumn_name, column_array.getData()); - - return ColumnArray::create(subcolumn, column_array.getOffsetsPtr()); -} - -SerializationPtr DataTypeArray::getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const -{ - return getSubcolumnSerializationImpl(subcolumn_name, base_serialization_getter, 0); -} - -SerializationPtr DataTypeArray::getSubcolumnSerializationImpl( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter, size_t level) const -{ - if (subcolumn_name == "size" + std::to_string(level)) - return std::make_shared(base_serialization_getter(DataTypeUInt64()), subcolumn_name, false); - - SerializationPtr subcolumn; - if (const auto * nested_array = typeid_cast(nested.get())) - subcolumn = nested_array->getSubcolumnSerializationImpl(subcolumn_name, base_serialization_getter, level + 1); - else - subcolumn = nested->getSubcolumnSerialization(subcolumn_name, base_serialization_getter); - - return std::make_shared(subcolumn); -} - SerializationPtr DataTypeArray::doGetDefaultSerialization() const { return std::make_shared(nested->getDefaultSerialization()); diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index c720a15d798..564dbba8503 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -54,23 +54,12 @@ public: return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion(); } - DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - SerializationPtr getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; - SerializationPtr doGetDefaultSerialization() const override; const DataTypePtr & getNestedType() const { return nested; } /// 1 for plain array, 2 for array of arrays and so on. size_t getNumberOfDimensions() const; - -private: - ColumnPtr getSubcolumnImpl(const String & subcolumn_name, const IColumn & column, size_t level) const; - DataTypePtr tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const; - SerializationPtr getSubcolumnSerializationImpl( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter, size_t level) const; }; } diff --git a/src/DataTypes/DataTypeDate.cpp b/src/DataTypes/DataTypeDate.cpp index 0df2e329702..ee4b0065e59 100644 --- a/src/DataTypes/DataTypeDate.cpp +++ b/src/DataTypes/DataTypeDate.cpp @@ -1,14 +1,7 @@ -#include -#include - -#include #include #include #include -#include - - namespace DB { diff --git a/src/DataTypes/DataTypeDateTime.cpp b/src/DataTypes/DataTypeDateTime.cpp index 4284c9ae4bd..c7722e1c1d9 100644 --- a/src/DataTypes/DataTypeDateTime.cpp +++ b/src/DataTypes/DataTypeDateTime.cpp @@ -1,28 +1,12 @@ #include #include -#include -#include -#include -#include -#include #include -#include #include -#include -#include -#include namespace DB { -TimezoneMixin::TimezoneMixin(const String & time_zone_name) - : has_explicit_time_zone(!time_zone_name.empty()), - time_zone(DateLUT::instance(time_zone_name)), - utc_time_zone(DateLUT::instance("UTC")) -{ -} - DataTypeDateTime::DataTypeDateTime(const String & time_zone_name) : TimezoneMixin(time_zone_name) { @@ -52,7 +36,7 @@ bool DataTypeDateTime::equals(const IDataType & rhs) const SerializationPtr DataTypeDateTime::doGetDefaultSerialization() const { - return std::make_shared(time_zone, utc_time_zone); + return std::make_shared(*this); } } diff --git a/src/DataTypes/DataTypeDateTime.h b/src/DataTypes/DataTypeDateTime.h index 926d529a5d8..57052144216 100644 --- a/src/DataTypes/DataTypeDateTime.h +++ b/src/DataTypes/DataTypeDateTime.h @@ -2,33 +2,11 @@ #include #include - -class DateLUTImpl; +#include namespace DB { -/** Mixin-class that manages timezone info for timezone-aware DateTime implementations - * - * Must be used as a (second) base for class implementing IDateType-interface. - */ -class TimezoneMixin -{ -public: - explicit TimezoneMixin(const String & time_zone_name = ""); - TimezoneMixin(const TimezoneMixin &) = default; - - const DateLUTImpl & getTimeZone() const { return time_zone; } - bool hasExplicitTimeZone() const { return has_explicit_time_zone; } - -protected: - /// true if time zone name was provided in data type parameters, false if it's using default time zone. - bool has_explicit_time_zone; - - const DateLUTImpl & time_zone; - const DateLUTImpl & utc_time_zone; -}; - /** DateTime stores time as unix timestamp. * The value itself is independent of time zone. * diff --git a/src/DataTypes/DataTypeDateTime64.cpp b/src/DataTypes/DataTypeDateTime64.cpp index bde7bebf455..4fa1569f0e8 100644 --- a/src/DataTypes/DataTypeDateTime64.cpp +++ b/src/DataTypes/DataTypeDateTime64.cpp @@ -1,19 +1,7 @@ #include #include - -#include -#include -#include -#include -#include -#include #include -#include #include -#include -#include -#include - #include #include @@ -65,7 +53,7 @@ bool DataTypeDateTime64::equals(const IDataType & rhs) const SerializationPtr DataTypeDateTime64::doGetDefaultSerialization() const { - return std::make_shared(time_zone, utc_time_zone, scale); + return std::make_shared(scale, *this); } } diff --git a/src/DataTypes/DataTypeDecimalBase.cpp b/src/DataTypes/DataTypeDecimalBase.cpp index f4c28088c48..62218694924 100644 --- a/src/DataTypes/DataTypeDecimalBase.cpp +++ b/src/DataTypes/DataTypeDecimalBase.cpp @@ -1,15 +1,5 @@ #include - -#include -#include -#include -#include -#include -#include #include -#include -#include - #include namespace DB diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index c86dd9d0b33..b659d92e3f4 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index a40592ba023..48034a31707 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -1,22 +1,12 @@ #include -#include -#include #include #include #include -#include -#include -#include -#include - #include #include -#include -#include - namespace DB { diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 5acf498c9fc..41de17982aa 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -1,9 +1,7 @@ #include #include #include -#include #include -#include #include #include #include @@ -11,14 +9,7 @@ #include #include #include -#include -#include -#include -#include -#include -#include #include -#include #include @@ -84,27 +75,6 @@ std::string DataTypeMap::doGetName() const return s.str(); } -static const IColumn & extractNestedColumn(const IColumn & column) -{ - return assert_cast(column).getNestedColumn(); -} - -DataTypePtr DataTypeMap::tryGetSubcolumnType(const String & subcolumn_name) const -{ - return nested->tryGetSubcolumnType(subcolumn_name); -} - -ColumnPtr DataTypeMap::getSubcolumn(const String & subcolumn_name, const IColumn & column) const -{ - return nested->getSubcolumn(subcolumn_name, extractNestedColumn(column)); -} - -SerializationPtr DataTypeMap::getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const -{ - return nested->getSubcolumnSerialization(subcolumn_name, base_serialization_getter); -} - MutableColumnPtr DataTypeMap::createColumn() const { return ColumnMap::create(nested->createColumn()); diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 09b8448885a..04377f85cfb 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -32,11 +32,6 @@ public: bool canBeInsideNullable() const override { return false; } - DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - SerializationPtr getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; - MutableColumnPtr createColumn() const override; Field getDefault() const override; diff --git a/src/DataTypes/DataTypeNested.cpp b/src/DataTypes/DataTypeNested.cpp index eba1bba5dfe..fe7cd515c81 100644 --- a/src/DataTypes/DataTypeNested.cpp +++ b/src/DataTypes/DataTypeNested.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include diff --git a/src/DataTypes/DataTypeNothing.cpp b/src/DataTypes/DataTypeNothing.cpp index 388a65754b5..c2b552035a0 100644 --- a/src/DataTypes/DataTypeNothing.cpp +++ b/src/DataTypes/DataTypeNothing.cpp @@ -1,10 +1,7 @@ -#include #include #include #include #include -#include -#include namespace DB diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 3820a320c6d..b354b1278be 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -1,17 +1,9 @@ #include #include -#include #include #include -#include #include #include -#include -#include -#include -#include -#include -#include #include #include #include @@ -63,32 +55,6 @@ bool DataTypeNullable::equals(const IDataType & rhs) const return rhs.isNullable() && nested_data_type->equals(*static_cast(rhs).nested_data_type); } -DataTypePtr DataTypeNullable::tryGetSubcolumnType(const String & subcolumn_name) const -{ - if (subcolumn_name == "null") - return std::make_shared(); - - return nested_data_type->tryGetSubcolumnType(subcolumn_name); -} - -ColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, const IColumn & column) const -{ - const auto & column_nullable = assert_cast(column); - if (subcolumn_name == "null") - return column_nullable.getNullMapColumnPtr(); - - return nested_data_type->getSubcolumn(subcolumn_name, column_nullable.getNestedColumn()); -} - -SerializationPtr DataTypeNullable::getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const -{ - if (subcolumn_name == "null") - return std::make_shared(base_serialization_getter(DataTypeUInt8()), subcolumn_name, false); - - return nested_data_type->getSubcolumnSerialization(subcolumn_name, base_serialization_getter); -} - SerializationPtr DataTypeNullable::doGetDefaultSerialization() const { return std::make_shared(nested_data_type->getDefaultSerialization()); diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 1557179d072..1a54d0de611 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -41,11 +41,6 @@ public: bool onlyNull() const override; bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); } - DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - SerializationPtr getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; - const DataTypePtr & getNestedType() const { return nested_data_type; } private: SerializationPtr doGetDefaultSerialization() const override; diff --git a/src/DataTypes/DataTypeNumberBase.cpp b/src/DataTypes/DataTypeNumberBase.cpp index a73d591654a..f668a4c522e 100644 --- a/src/DataTypes/DataTypeNumberBase.cpp +++ b/src/DataTypes/DataTypeNumberBase.cpp @@ -1,13 +1,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include namespace DB diff --git a/src/DataTypes/DataTypeString.cpp b/src/DataTypes/DataTypeString.cpp index 41ae578a70f..84610557d21 100644 --- a/src/DataTypes/DataTypeString.cpp +++ b/src/DataTypes/DataTypeString.cpp @@ -1,14 +1,6 @@ -#include - #include -#include - -#include -#include - #include -#include #include #include #include @@ -16,15 +8,6 @@ #include #include -#include -#include -#include - -#ifdef __SSE2__ - #include -#endif - - namespace DB { diff --git a/src/DataTypes/DataTypeString.h b/src/DataTypes/DataTypeString.h index 0fc38e9c6f0..fd674505bc0 100644 --- a/src/DataTypes/DataTypeString.h +++ b/src/DataTypes/DataTypeString.h @@ -1,7 +1,5 @@ #pragma once -#include - #include diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index aa0a57c636e..6bca7f2bac2 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -3,20 +3,17 @@ #include #include #include -#include #include #include #include #include -#include +#include #include #include #include -#include #include #include #include -#include #include #include @@ -107,11 +104,6 @@ static inline IColumn & extractElementColumn(IColumn & column, size_t idx) return assert_cast(column).getColumn(idx); } -static inline const IColumn & extractElementColumn(const IColumn & column, size_t idx) -{ - return assert_cast(column).getColumn(idx); -} - template static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl) { @@ -234,74 +226,6 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const return res; } -template -auto DataTypeTuple::getSubcolumnEntity(const String & subcolumn_name, - const OnSuccess & on_success, const OnContinue & on_continue) const -{ - using ReturnType = decltype(on_success(0)); - for (size_t i = 0; i < names.size(); ++i) - { - if (startsWith(subcolumn_name, names[i])) - { - size_t name_length = names[i].size(); - - if (subcolumn_name.size() == name_length) - return on_success(i); - - if (subcolumn_name[name_length] == '.') - return on_continue(i, subcolumn_name.substr(name_length + 1)); - } - } - - return ReturnType{}; -} - -DataTypePtr DataTypeTuple::tryGetSubcolumnType(const String & subcolumn_name) const -{ - if (subcolumn_name == MAIN_SUBCOLUMN_NAME) - return shared_from_this(); - - auto on_success = [&](size_t pos) { return elems[pos]; }; - auto on_continue = [&](size_t pos, const String & next_subcolumn) { return elems[pos]->tryGetSubcolumnType(next_subcolumn); }; - - return getSubcolumnEntity(subcolumn_name, on_success, on_continue); -} - -ColumnPtr DataTypeTuple::getSubcolumn(const String & subcolumn_name, const IColumn & column) const -{ - auto on_success = [&](size_t pos) { return extractElementColumn(column, pos).getPtr(); }; - auto on_continue = [&](size_t pos, const String & next_subcolumn) - { - return elems[pos]->getSubcolumn(next_subcolumn, extractElementColumn(column, pos)); - }; - - if (auto subcolumn = getSubcolumnEntity(subcolumn_name, on_success, on_continue)) - return subcolumn; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); -} - -SerializationPtr DataTypeTuple::getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const -{ - auto on_success = [&](size_t pos) - { - return std::make_shared(base_serialization_getter(*elems[pos]), names[pos]); - }; - - auto on_continue = [&](size_t pos, const String & next_subcolumn) - { - auto next_serialization = elems[pos]->getSubcolumnSerialization(next_subcolumn, base_serialization_getter); - return std::make_shared(next_serialization, names[pos]); - }; - - if (auto serialization = getSubcolumnEntity(subcolumn_name, on_success, on_continue)) - return serialization; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); -} - - SerializationPtr DataTypeTuple::doGetDefaultSerialization() const { SerializationTuple::ElementSerializations serializations(elems.size()); @@ -310,7 +234,7 @@ SerializationPtr DataTypeTuple::doGetDefaultSerialization() const { String elem_name = use_explicit_names ? names[i] : toString(i + 1); auto serialization = elems[i]->getDefaultSerialization(); - serializations[i] = std::make_shared(serialization, elem_name); + serializations[i] = std::make_shared(serialization, elem_name); } return std::make_shared(std::move(serializations), use_explicit_names); @@ -325,7 +249,7 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con String elem_name = use_explicit_names ? names[i] : toString(i + 1); auto subcolumn_name = Nested::concatenateName(column_name, elem_name); auto serializaion = elems[i]->getSerialization(subcolumn_name, callback); - serializations[i] = std::make_shared(serializaion, elem_name); + serializations[i] = std::make_shared(serializaion, elem_name); } return std::make_shared(std::move(serializations), use_explicit_names); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index e572b23f987..8dae8b7765b 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -52,16 +52,11 @@ public: size_t getMaximumSizeOfValueInMemory() const override; size_t getSizeOfValueInMemory() const override; - DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const override; - SerializationPtr getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; - SerializationPtr doGetDefaultSerialization() const override; + const DataTypePtr & getElement(size_t i) const { return elems[i]; } const DataTypes & getElements() const { return elems; } const Strings & getElementNames() const { return names; } @@ -69,11 +64,6 @@ public: bool haveExplicitNames() const { return have_explicit_names; } bool serializeNames() const { return serialize_names; } - -private: - template - auto getSubcolumnEntity(const String & subcolumn_name, - const OnSuccess & on_success, const OnContinue & on_continue) const; }; } diff --git a/src/DataTypes/DataTypesDecimal.cpp b/src/DataTypes/DataTypesDecimal.cpp index 9f7320197c8..f0fbd6cab26 100644 --- a/src/DataTypes/DataTypesDecimal.cpp +++ b/src/DataTypes/DataTypesDecimal.cpp @@ -1,16 +1,13 @@ #include #include -#include #include #include #include #include #include #include -#include #include -#include #include diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 93bb1757a4d..669876c792d 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -11,7 +10,6 @@ #include #include #include -#include namespace DB @@ -65,12 +63,40 @@ size_t IDataType::getSizeOfValueInMemory() const throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR); } +void IDataType::forEachSubcolumn( + const SubcolumnCallback & callback, + const SerializationPtr & serialization, + const DataTypePtr & type, + const ColumnPtr & column) +{ + ISerialization::StreamCallback callback_with_data = [&](const auto & subpath) + { + for (size_t i = 0; i < subpath.size(); ++i) + { + if (!subpath[i].visited && ISerialization::hasSubcolumnForPath(subpath, i + 1)) + { + auto name = ISerialization::getSubcolumnNameForStream(subpath, i + 1); + auto data = ISerialization::createFromPath(subpath, i); + callback(subpath, name, data); + } + subpath[i].visited = true; + } + }; + + ISerialization::SubstreamPath path; + serialization->enumerateStreams(path, callback_with_data, type, column); +} + DataTypePtr IDataType::tryGetSubcolumnType(const String & subcolumn_name) const { - if (subcolumn_name == MAIN_SUBCOLUMN_NAME) - return shared_from_this(); + DataTypePtr res; + forEachSubcolumn([&](const auto &, const auto & name, const auto & data) + { + if (name == subcolumn_name) + res = data.type; + }, getDefaultSerialization(), getPtr(), nullptr); - return nullptr; + return res; } DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const @@ -82,42 +108,43 @@ DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } -ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const IColumn &) const +SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const { + SerializationPtr res; + forEachSubcolumn([&](const auto &, const auto & name, const auto & data) + { + if (name == subcolumn_name) + res = data.serialization; + }, serialization, nullptr, nullptr); + + if (res) + return res; + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } -void IDataType::forEachSubcolumn(const SubcolumnCallback & callback) const +ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const { - NameSet set; - getDefaultSerialization()->enumerateStreams([&, this](const ISerialization::SubstreamPath & substream_path) + ColumnPtr res; + forEachSubcolumn([&](const auto &, const auto & name, const auto & data) { - ISerialization::SubstreamPath new_path; - /// Iterate over path to try to get intermediate subcolumns for complex nested types. - for (const auto & elem : substream_path) - { - new_path.push_back(elem); - auto name = ISerialization::getSubcolumnNameForStream(new_path); - auto type = tryGetSubcolumnType(name); + if (name == subcolumn_name) + res = data.column; + }, getDefaultSerialization(), nullptr, column); - /// Subcolumn names may repeat among several substream paths. - if (!name.empty() && type && !set.count(name)) - { - callback(name, type, substream_path); - set.insert(name); - } - } - }); + if (res) + return res; + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } Names IDataType::getSubcolumnNames() const { Names res; - forEachSubcolumn([&](const auto & name, const auto &, const auto &) + forEachSubcolumn([&](const auto &, const auto & name, const auto &) { res.push_back(name); - }); - + }, getDefaultSerialization(), nullptr, nullptr); return res; } @@ -144,24 +171,14 @@ SerializationPtr IDataType::getDefaultSerialization() const return doGetDefaultSerialization(); } -SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const BaseSerializationGetter &) const -{ - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); -} - // static SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, const IDataType::StreamExistenceCallback & callback) { if (column.isSubcolumn()) { - /// Wrap to custom serialization deepest subcolumn, which is represented in non-complex type. - auto base_serialization_getter = [&](const IDataType & subcolumn_type) - { - return subcolumn_type.getSerialization(column.name, callback); - }; - const auto & type_in_storage = column.getTypeInStorage(); - return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter); + auto default_serialization = type_in_storage->getDefaultSerialization(); + return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), default_serialization); } return column.type->getSerialization(column.name, callback); @@ -172,21 +189,4 @@ SerializationPtr IDataType::getSerialization(const String &, const StreamExisten return getDefaultSerialization(); } -DataTypePtr IDataType::getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const -{ - auto type = tryGetSubcolumnType(ISerialization::getSubcolumnNameForStream(substream_path)); - if (type) - return type->getSubcolumnType(MAIN_SUBCOLUMN_NAME); - - return getSubcolumnType(MAIN_SUBCOLUMN_NAME); -} - -void IDataType::enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath & path) const -{ - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) - { - callback(substream_path, *getTypeForSubstream(substream_path)); - }, path); -} - } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 360bf9f16e0..a53fdac797f 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -70,19 +70,31 @@ public: return doGetName(); } + DataTypePtr getPtr() const { return shared_from_this(); } + /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; /// Data type id. It's used for runtime type checks. virtual TypeIndex getTypeId() const = 0; - static constexpr auto MAIN_SUBCOLUMN_NAME = "__main"; - virtual DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const; + DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const; DataTypePtr getSubcolumnType(const String & subcolumn_name) const; - virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const; - using SubcolumnCallback = std::function; - void forEachSubcolumn(const SubcolumnCallback & callback) const; + SerializationPtr getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const; + ColumnPtr getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const; + + using SubcolumnCallback = std::function; + + static void forEachSubcolumn( + const SubcolumnCallback & callback, + const SerializationPtr & serialization, + const DataTypePtr & type, + const ColumnPtr & column); + Names getSubcolumnNames() const; /// Returns default serialization of data type. @@ -93,7 +105,6 @@ public: /// one of serialization types, that serialization will be chosen for reading. /// If callback always returned false, the default serialization will be chosen. using StreamExistenceCallback = std::function; - using BaseSerializationGetter = std::function; /// Chooses serialization for reading of one column or subcolumns by /// checking existence of substreams using callback. @@ -103,22 +114,10 @@ public: virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const; - /// Returns serialization wrapper for reading one particular subcolumn of data type. - virtual SerializationPtr getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; - - using StreamCallbackWithType = std::function; - - void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath & path) const; - void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath && path) const { enumerateStreams(serialization, callback, path); } - void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback) const { enumerateStreams(serialization, callback, {}); } - protected: virtual String doGetName() const { return getFamilyName(); } virtual SerializationPtr doGetDefaultSerialization() const = 0; - DataTypePtr getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const; - public: /** Create empty column for corresponding type. */ diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 7077c5bfa14..a6d9185429c 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -17,30 +18,11 @@ namespace ErrorCodes String ISerialization::Substream::toString() const { - switch (type) - { - case ArrayElements: - return "ArrayElements"; - case ArraySizes: - return "ArraySizes"; - case NullableElements: - return "NullableElements"; - case NullMap: - return "NullMap"; - case TupleElement: - return "TupleElement(" + tuple_element_name + ", " - + std::to_string(escape_tuple_delimiter) + ")"; - case DictionaryKeys: - return "DictionaryKeys"; - case DictionaryIndexes: - return "DictionaryIndexes"; - case SparseElements: - return "SparseElements"; - case SparseOffsets: - return "SparseOffsets"; - } + if (type == TupleElement) + return fmt::format("TupleElement({}, escape_tuple_delimiter={})", + tuple_element_name, escape_tuple_delimiter ? "true" : "false"); - __builtin_unreachable(); + return String(magic_enum::enum_name(type)); } String ISerialization::SubstreamPath::toString() const @@ -57,9 +39,21 @@ String ISerialization::SubstreamPath::toString() const return wb.str(); } +void ISerialization::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const +{ + path.push_back(Substream::Regular); + path.back().data = {type, column, getPtr(), nullptr}; + callback(path); + path.pop_back(); +} + void ISerialization::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { - callback(path); + enumerateStreams(path, callback, nullptr, nullptr); } void ISerialization::serializeBinaryBulk(const IColumn & column, WriteBuffer &, size_t, size_t) const @@ -104,40 +98,48 @@ void ISerialization::deserializeBinaryBulkWithMultipleStreams( } } +namespace +{ + +using SubstreamIterator = ISerialization::SubstreamPath::const_iterator; + static String getNameForSubstreamPath( String stream_name, - const ISerialization::SubstreamPath & path, + SubstreamIterator begin, + SubstreamIterator end, bool escape_tuple_delimiter) { using Substream = ISerialization::Substream; size_t array_level = 0; - for (const auto & elem : path) + for (auto it = begin; it != end; ++it) { - if (elem.type == Substream::NullMap) + if (it->type == Substream::NullMap) stream_name += ".null"; - else if (elem.type == Substream::ArraySizes) + else if (it->type == Substream::ArraySizes) stream_name += ".size" + toString(array_level); - else if (elem.type == Substream::ArrayElements) + else if (it->type == Substream::ArrayElements) ++array_level; - else if (elem.type == Substream::DictionaryKeys) + else if (it->type == Substream::DictionaryKeys) stream_name += ".dict"; - else if (elem.type == Substream::SparseOffsets) + else if (it->type == Substream::SparseOffsets) stream_name += ".sparse.idx"; - else if (elem.type == Substream::TupleElement) + else if (it->type == Substream::TupleElement) { /// For compatibility reasons, we use %2E (escaped dot) instead of dot. /// Because nested data may be represented not by Array of Tuple, /// but by separate Array columns with names in a form of a.b, /// and name is encoded as a whole. - stream_name += (escape_tuple_delimiter && elem.escape_tuple_delimiter ? - escapeForFileName(".") : ".") + escapeForFileName(elem.tuple_element_name); + stream_name += (escape_tuple_delimiter && it->escape_tuple_delimiter ? + escapeForFileName(".") : ".") + escapeForFileName(it->tuple_element_name); } } return stream_name; } +} + String ISerialization::getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path) { return getFileNameForStream(column.getNameInStorage(), path); @@ -152,12 +154,17 @@ String ISerialization::getFileNameForStream(const String & name_in_storage, cons else stream_name = escapeForFileName(name_in_storage); - return getNameForSubstreamPath(std::move(stream_name), path, true); + return getNameForSubstreamPath(std::move(stream_name), path.begin(), path.end(), true); } String ISerialization::getSubcolumnNameForStream(const SubstreamPath & path) { - auto subcolumn_name = getNameForSubstreamPath("", path, false); + return getSubcolumnNameForStream(path, path.size()); +} + +String ISerialization::getSubcolumnNameForStream(const SubstreamPath & path, size_t prefix_len) +{ + auto subcolumn_name = getNameForSubstreamPath("", path.begin(), path.begin() + prefix_len, false); if (!subcolumn_name.empty()) subcolumn_name = subcolumn_name.substr(1); // It starts with a dot. @@ -195,4 +202,44 @@ bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path) return true; } +size_t ISerialization::getArrayLevel(const SubstreamPath & path) +{ + size_t level = 0; + for (const auto & elem : path) + level += elem.type == Substream::ArrayElements; + return level; } + +bool ISerialization::hasSubcolumnForPath(const SubstreamPath & path, size_t prefix_len) +{ + if (prefix_len == 0 || prefix_len > path.size()) + return false; + + size_t last_elem = prefix_len - 1; + return path[last_elem].type == Substream::NullMap + || path[last_elem].type == Substream::TupleElement + || path[last_elem].type == Substream::ArraySizes; +} + +ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath & path, size_t prefix_len) +{ + assert(prefix_len < path.size()); + + SubstreamData res = path[prefix_len].data; + res.creator.reset(); + for (ssize_t i = static_cast(prefix_len) - 1; i >= 0; --i) + { + const auto & creator = path[i].data.creator; + if (creator) + { + res.type = res.type ? creator->create(res.type) : res.type; + res.serialization = res.serialization ? creator->create(res.serialization) : res.serialization; + res.column = res.column ? creator->create(res.column) : res.column; + } + } + + return res; +} + +} + diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index f1d82a2000a..7562cfcb9a0 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -2,35 +2,39 @@ #include #include +#include +#include #include #include namespace DB { -class IDataType; - class ReadBuffer; class WriteBuffer; class ProtobufReader; class ProtobufWriter; -class IColumn; -using ColumnPtr = COW::Ptr; -using MutableColumnPtr = COW::MutablePtr; +class IDataType; +using DataTypePtr = std::shared_ptr; + +class ISerialization; +using SerializationPtr = std::shared_ptr; class Field; struct FormatSettings; struct NameAndTypePair; -class ISerialization +class ISerialization : private boost::noncopyable, public std::enable_shared_from_this { public: ISerialization() = default; virtual ~ISerialization() = default; + SerializationPtr getPtr() const { return shared_from_this(); } + /** Binary serialization for range of values in column - for writing to disk/network, etc. * * Some data types are represented in multiple streams while being serialized. @@ -54,6 +58,24 @@ public: * Default implementations of ...WithMultipleStreams methods will call serializeBinaryBulk, deserializeBinaryBulk for single stream. */ + struct ISubcolumnCreator + { + virtual DataTypePtr create(const DataTypePtr & prev) const = 0; + virtual SerializationPtr create(const SerializationPtr & prev) const = 0; + virtual ColumnPtr create(const ColumnPtr & prev) const = 0; + virtual ~ISubcolumnCreator() = default; + }; + + using SubcolumnCreatorPtr = std::shared_ptr; + + struct SubstreamData + { + DataTypePtr type; + ColumnPtr column; + SerializationPtr serialization; + SubcolumnCreatorPtr creator; + }; + struct Substream { enum Type @@ -71,7 +93,10 @@ public: SparseElements, SparseOffsets, + + Regular, }; + Type type; /// Index of tuple element, starting at 1 or name. @@ -80,6 +105,12 @@ public: /// Do we need to escape a dot in filenames for tuple elements. bool escape_tuple_delimiter = true; + /// Data for current substream. + SubstreamData data; + + /// Flag, that may help to traverse substream paths. + mutable bool visited = false; + Substream(Type type_) : type(type_) {} String toString() const; @@ -96,7 +127,13 @@ public: using StreamCallback = std::function; - virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const; + virtual void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const; + + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const; void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } @@ -249,11 +286,16 @@ public: static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); static String getFileNameForStream(const String & name_in_storage, const SubstreamPath & path); static String getSubcolumnNameForStream(const SubstreamPath & path); + static String getSubcolumnNameForStream(const SubstreamPath & path, size_t prefix_len); static void addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column); static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path); static bool isSpecialCompressionAllowed(const SubstreamPath & path); + static size_t getArrayLevel(const SubstreamPath & path); + + static bool hasSubcolumnForPath(const SubstreamPath & path, size_t prefix_len); + static SubstreamData createFromPath(const SubstreamPath & path, size_t prefix_len); }; using SerializationPtr = std::shared_ptr; diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 925ba0b9e74..2339f23853e 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -1,7 +1,6 @@ #include #include -#include #include diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 70a72c51e78..4ccee54c294 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -1,5 +1,8 @@ #include #include +#include +#include +#include #include #include #include @@ -177,16 +180,53 @@ ColumnPtr arrayOffsetsToSizes(const IColumn & column) return column_sizes; } - -void SerializationArray::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +DataTypePtr SerializationArray::SubcolumnCreator::create(const DataTypePtr & prev) const { - path.push_back(Substream::ArraySizes); - callback(path); - path.back() = Substream::ArrayElements; - nested->enumerateStreams(callback, path); - path.pop_back(); + return std::make_shared(prev); } +SerializationPtr SerializationArray::SubcolumnCreator::create(const SerializationPtr & prev) const +{ + return std::make_shared(prev); +} + +ColumnPtr SerializationArray::SubcolumnCreator::create(const ColumnPtr & prev) const +{ + return ColumnArray::create(prev, offsets); +} + +void SerializationArray::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const +{ + const auto * type_array = type ? &assert_cast(*type) : nullptr; + const auto * column_array = column ? &assert_cast(*column) : nullptr; + auto offsets_column = column_array ? column_array->getOffsetsPtr() : nullptr; + + path.push_back(Substream::ArraySizes); + path.back().data = + { + type ? std::make_shared() : nullptr, + offsets_column ? arrayOffsetsToSizes(*offsets_column) : nullptr, + std::make_shared( + std::make_shared>(), + "size" + std::to_string(getArrayLevel(path)), false), + nullptr, + }; + + callback(path); + + path.back() = Substream::ArrayElements; + path.back().data = {type, column, getPtr(), std::make_shared(offsets_column)}; + + auto next_type = type_array ? type_array->getNestedType() : nullptr; + auto next_column = column_array ? column_array->getDataPtr() : nullptr; + + nested->enumerateStreams(path, callback, next_type, next_column); + path.pop_back(); +} void SerializationArray::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 71037090a48..83045d4c033 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -35,7 +35,11 @@ public: * This is necessary, because when implementing nested structures, several arrays can have common sizes. */ - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, @@ -62,6 +66,18 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; + +private: + struct SubcolumnCreator : public ISubcolumnCreator + { + const ColumnPtr offsets; + + SubcolumnCreator(const ColumnPtr & offsets_) : offsets(offsets_) {} + + DataTypePtr create(const DataTypePtr & prev) const override; + SerializationPtr create(const SerializationPtr & prev) const override; + ColumnPtr create(const ColumnPtr & prev) const override; + }; }; ColumnPtr arrayOffsetsToSizes(const IColumn & column); diff --git a/src/DataTypes/Serializations/SerializationDate32.cpp b/src/DataTypes/Serializations/SerializationDate32.cpp index e43edbac592..ff1bb9b2c30 100644 --- a/src/DataTypes/Serializations/SerializationDate32.cpp +++ b/src/DataTypes/Serializations/SerializationDate32.cpp @@ -8,6 +8,7 @@ namespace DB { + void SerializationDate32::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { writeDateText(ExtendedDayNum(assert_cast(column).getData()[row_num]), ostr); diff --git a/src/DataTypes/Serializations/SerializationDateTime.cpp b/src/DataTypes/Serializations/SerializationDateTime.cpp index b93c69203cb..0e3b60f3772 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime.cpp @@ -32,9 +32,8 @@ inline void readText(time_t & x, ReadBuffer & istr, const FormatSettings & setti } -SerializationDateTime::SerializationDateTime( - const DateLUTImpl & time_zone_, const DateLUTImpl & utc_time_zone_) - : time_zone(time_zone_), utc_time_zone(utc_time_zone_) +SerializationDateTime::SerializationDateTime(const TimezoneMixin & time_zone_) + : TimezoneMixin(time_zone_) { } diff --git a/src/DataTypes/Serializations/SerializationDateTime.h b/src/DataTypes/Serializations/SerializationDateTime.h index 8cf57ddef89..75334592422 100644 --- a/src/DataTypes/Serializations/SerializationDateTime.h +++ b/src/DataTypes/Serializations/SerializationDateTime.h @@ -1,20 +1,17 @@ #pragma once #include +#include class DateLUTImpl; namespace DB { -class SerializationDateTime final : public SerializationNumber +class SerializationDateTime final : public SerializationNumber, public TimezoneMixin { -private: - const DateLUTImpl & time_zone; - const DateLUTImpl & utc_time_zone; - public: - SerializationDateTime(const DateLUTImpl & time_zone_, const DateLUTImpl & utc_time_zone_); + SerializationDateTime(const TimezoneMixin & time_zone_); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; diff --git a/src/DataTypes/Serializations/SerializationDateTime64.cpp b/src/DataTypes/Serializations/SerializationDateTime64.cpp index 8d446d3b9ad..613422f21fb 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.cpp +++ b/src/DataTypes/Serializations/SerializationDateTime64.cpp @@ -17,9 +17,9 @@ namespace DB { SerializationDateTime64::SerializationDateTime64( - const DateLUTImpl & time_zone_, const DateLUTImpl & utc_time_zone_, UInt32 scale_) + UInt32 scale_, const TimezoneMixin & time_zone_) : SerializationDecimalBase(DecimalUtils::max_precision, scale_) - , time_zone(time_zone_), utc_time_zone(utc_time_zone_) + , TimezoneMixin(time_zone_) { } diff --git a/src/DataTypes/Serializations/SerializationDateTime64.h b/src/DataTypes/Serializations/SerializationDateTime64.h index c36649daef1..1679170b36f 100644 --- a/src/DataTypes/Serializations/SerializationDateTime64.h +++ b/src/DataTypes/Serializations/SerializationDateTime64.h @@ -1,20 +1,17 @@ #pragma once #include +#include class DateLUTImpl; namespace DB { -class SerializationDateTime64 final : public SerializationDecimalBase +class SerializationDateTime64 final : public SerializationDecimalBase, public TimezoneMixin { -private: - const DateLUTImpl & time_zone; - const DateLUTImpl & utc_time_zone; - public: - SerializationDateTime64(const DateLUTImpl & time_zone_, const DateLUTImpl & utc_time_zone_, UInt32 scale_); + SerializationDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index e9bb62f74c5..7a86d5413b2 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -40,11 +40,27 @@ SerializationLowCardinality::SerializationLowCardinality(const DataTypePtr & dic { } -void SerializationLowCardinality::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void SerializationLowCardinality::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const { + const auto * column_lc = column ? &getColumnLowCardinality(*column) : nullptr; + + SubstreamData data; + data.type = type ? dictionary_type : nullptr; + data.column = column_lc ? column_lc->getDictionary().getNestedColumn() : nullptr; + data.serialization = dict_inner_serialization; + path.push_back(Substream::DictionaryKeys); - dict_inner_serialization->enumerateStreams(callback, path); + path.back().data = data; + + dict_inner_serialization->enumerateStreams(path, callback, data.type, data.column); + path.back() = Substream::DictionaryIndexes; + path.back().data = {type, column, getPtr(), nullptr}; + callback(path); path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index e9ca0349e38..f82b35a52d5 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -17,7 +17,11 @@ private: public: SerializationLowCardinality(const DataTypePtr & dictionary_type); - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index e8446781f10..2e436070e1c 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -250,10 +251,16 @@ void SerializationMap::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c deserializeText(column, rb, settings); } - -void SerializationMap::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void SerializationMap::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const { - nested->enumerateStreams(callback, path); + auto next_type = type ? assert_cast(*type).getNestedType() : nullptr; + auto next_column = column ? assert_cast(*column).getNestedColumnPtr() : nullptr; + + nested->enumerateStreams(path, callback, next_type, next_column); } void SerializationMap::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 6523d5388d0..b6a003139ec 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -31,7 +31,11 @@ public: void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationTupleElement.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp similarity index 69% rename from src/DataTypes/Serializations/SerializationTupleElement.cpp rename to src/DataTypes/Serializations/SerializationNamed.cpp index 4b50810fcd6..4ef4d4527f8 100644 --- a/src/DataTypes/Serializations/SerializationTupleElement.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -1,18 +1,21 @@ -#include +#include namespace DB { -void SerializationTupleElement::enumerateStreams( +void SerializationNamed::enumerateStreams( + SubstreamPath & path, const StreamCallback & callback, - SubstreamPath & path) const + DataTypePtr type, + ColumnPtr column) const { addToPath(path); - nested_serialization->enumerateStreams(callback, path); + path.back().data = {type, column, getPtr(), std::make_shared(name, escape_delimiter)}; + nested_serialization->enumerateStreams(path, callback, type, column); path.pop_back(); } -void SerializationTupleElement::serializeBinaryBulkStatePrefix( +void SerializationNamed::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -21,7 +24,7 @@ void SerializationTupleElement::serializeBinaryBulkStatePrefix( settings.path.pop_back(); } -void SerializationTupleElement::serializeBinaryBulkStateSuffix( +void SerializationNamed::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -30,7 +33,7 @@ void SerializationTupleElement::serializeBinaryBulkStateSuffix( settings.path.pop_back(); } -void SerializationTupleElement::deserializeBinaryBulkStatePrefix( +void SerializationNamed::deserializeBinaryBulkStatePrefix( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { @@ -39,7 +42,7 @@ void SerializationTupleElement::deserializeBinaryBulkStatePrefix( settings.path.pop_back(); } -void SerializationTupleElement::serializeBinaryBulkWithMultipleStreams( +void SerializationNamed::serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, @@ -51,7 +54,7 @@ void SerializationTupleElement::serializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } -void SerializationTupleElement::deserializeBinaryBulkWithMultipleStreams( +void SerializationNamed::deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, size_t limit, DeserializeBinaryBulkSettings & settings, @@ -63,7 +66,7 @@ void SerializationTupleElement::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } -void SerializationTupleElement::addToPath(SubstreamPath & path) const +void SerializationNamed::addToPath(SubstreamPath & path) const { path.push_back(Substream::TupleElement); path.back().tuple_element_name = name; diff --git a/src/DataTypes/Serializations/SerializationTupleElement.h b/src/DataTypes/Serializations/SerializationNamed.h similarity index 59% rename from src/DataTypes/Serializations/SerializationTupleElement.h rename to src/DataTypes/Serializations/SerializationNamed.h index b85014c9e64..20dd15a20ba 100644 --- a/src/DataTypes/Serializations/SerializationTupleElement.h +++ b/src/DataTypes/Serializations/SerializationNamed.h @@ -5,14 +5,14 @@ namespace DB { -class SerializationTupleElement final : public SerializationWrapper +class SerializationNamed final : public SerializationWrapper { private: String name; bool escape_delimiter; public: - SerializationTupleElement(const SerializationPtr & nested_, const String & name_, bool escape_delimiter_ = true) + SerializationNamed(const SerializationPtr & nested_, const String & name_, bool escape_delimiter_ = true) : SerializationWrapper(nested_) , name(name_), escape_delimiter(escape_delimiter_) { @@ -21,11 +21,13 @@ public: const String & getElementName() const { return name; } void enumerateStreams( + SubstreamPath & path, const StreamCallback & callback, - SubstreamPath & path) const override; + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; void serializeBinaryBulkStateSuffix( @@ -51,6 +53,22 @@ public: SubstreamsCache * cache) const override; private: + struct SubcolumnCreator : public ISubcolumnCreator + { + const String name; + const bool escape_delimiter; + + SubcolumnCreator(const String & name_, bool escape_delimiter_) + : name(name_), escape_delimiter(escape_delimiter_) {} + + DataTypePtr create(const DataTypePtr & prev) const override { return prev; } + ColumnPtr create(const ColumnPtr & prev) const override { return prev; } + SerializationPtr create(const SerializationPtr & prev) const override + { + return std::make_shared(prev, name, escape_delimiter); + } + }; + void addToPath(SubstreamPath & path) const; }; diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index b607d5871d6..560a4812123 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -1,5 +1,8 @@ #include #include +#include +#include +#include #include #include @@ -20,15 +23,50 @@ namespace ErrorCodes extern const int CANNOT_READ_ALL_DATA; } -void SerializationNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +DataTypePtr SerializationNullable::SubcolumnCreator::create(const DataTypePtr & prev) const { - path.push_back(Substream::NullMap); - callback(path); - path.back() = Substream::NullableElements; - nested->enumerateStreams(callback, path); - path.pop_back(); + return std::make_shared(prev); } +SerializationPtr SerializationNullable::SubcolumnCreator::create(const SerializationPtr & prev) const +{ + return std::make_shared(prev); +} + +ColumnPtr SerializationNullable::SubcolumnCreator::create(const ColumnPtr & prev) const +{ + return ColumnNullable::create(prev, null_map); +} + +void SerializationNullable::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const +{ + const auto * type_nullable = type ? &assert_cast(*type) : nullptr; + const auto * column_nullable = column ? &assert_cast(*column) : nullptr; + + path.push_back(Substream::NullMap); + path.back().data = + { + type_nullable ? std::make_shared() : nullptr, + column_nullable ? column_nullable->getNullMapColumnPtr() : nullptr, + std::make_shared(std::make_shared>(), "null", false), + nullptr, + }; + + callback(path); + + path.back() = Substream::NullableElements; + path.back().data = {type, column, getPtr(), std::make_shared(path.back().data.column)}; + + auto next_type = type_nullable ? type_nullable->getNestedType() : nullptr; + auto next_column = column_nullable ? column_nullable->getNestedColumnPtr() : nullptr; + + nested->enumerateStreams(path, callback, next_type, next_column); + path.pop_back(); +} void SerializationNullable::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index b0b96c021d3..c39c4dd6573 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -13,7 +13,11 @@ private: public: SerializationNullable(const SerializationPtr & nested_) : nested(nested_) {} - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, @@ -80,6 +84,18 @@ public: static ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); template static ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); + +private: + struct SubcolumnCreator : public ISubcolumnCreator + { + const ColumnPtr null_map; + + SubcolumnCreator(const ColumnPtr & null_map_) : null_map(null_map_) {} + + DataTypePtr create(const DataTypePtr & prev) const override; + SerializationPtr create(const SerializationPtr & prev) const override; + ColumnPtr create(const ColumnPtr & prev) const override; + }; }; } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 1324c6b2b1a..33a90a4abf6 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -281,10 +282,22 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, }); } -void SerializationTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void SerializationTuple::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const { - for (const auto & elem : elems) - elem->enumerateStreams(callback, path); + const auto * type_tuple = type ? &assert_cast(*type) : nullptr; + const auto * column_tuple = column ? &assert_cast(*column) : nullptr; + + for (size_t i = 0; i < elems.size(); ++i) + { + auto next_type = type_tuple ? type_tuple->getElement(i) : nullptr; + auto next_column = column_tuple ? column_tuple->getColumnPtr(i) : nullptr; + + elems[i]->enumerateStreams(path, callback, next_type, next_column); + } } struct SerializeBinaryBulkStateTuple : public ISerialization::SerializeBinaryBulkState diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index 13668572fff..77f8de90c83 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { @@ -9,7 +9,7 @@ namespace DB class SerializationTuple final : public SimpleTextSerialization { public: - using ElementSerializationPtr = std::shared_ptr; + using ElementSerializationPtr = std::shared_ptr; using ElementSerializations = std::vector; SerializationTuple(const ElementSerializations & elems_, bool have_explicit_names_) @@ -31,7 +31,11 @@ public: /** Each sub-column in a tuple is serialized in separate stream. */ - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index f75c9a1dd8b..c0829ab1b26 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -4,9 +4,13 @@ namespace DB { -void SerializationWrapper::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void SerializationWrapper::enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const { - nested_serialization->enumerateStreams(callback, path); + nested_serialization->enumerateStreams(path, callback, type, column); } void SerializationWrapper::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 399d3b198b3..c48278d53db 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -16,7 +16,11 @@ protected: public: SerializationWrapper(const SerializationPtr & nested_serialization_) : nested_serialization(nested_serialization_) {} - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreams( + SubstreamPath & path, + const StreamCallback & callback, + DataTypePtr type, + ColumnPtr column) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h new file mode 100644 index 00000000000..e6e9f7a7989 --- /dev/null +++ b/src/DataTypes/TimezoneMixin.h @@ -0,0 +1,32 @@ +#pragma once +#include +#include + +class DateLUTImpl; + +/** Mixin-class that manages timezone info for timezone-aware DateTime implementations + * + * Must be used as a (second) base for class implementing IDateType/ISerialization-interface. + */ +class TimezoneMixin +{ +public: + TimezoneMixin(const TimezoneMixin &) = default; + + explicit TimezoneMixin(const String & time_zone_name = "") + : has_explicit_time_zone(!time_zone_name.empty()) + , time_zone(DateLUT::instance(time_zone_name)) + , utc_time_zone(DateLUT::instance("UTC")) + { + } + + const DateLUTImpl & getTimeZone() const { return time_zone; } + bool hasExplicitTimeZone() const { return has_explicit_time_zone; } + +protected: + /// true if time zone name was provided in data type parameters, false if it's using default time zone. + bool has_explicit_time_zone; + + const DateLUTImpl & time_zone; + const DateLUTImpl & utc_time_zone; +}; diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 2ebae17cd6b..00a2e4bc8bb 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -128,10 +128,10 @@ BlockIO InterpreterDescribeQuery::execute() { for (const auto & column : columns) { - column.type->forEachSubcolumn([&](const auto & name, const auto & type, const auto & path) + IDataType::forEachSubcolumn([&](const auto & path, const auto & name, const auto & data) { res_columns[0]->insert(Nested::concatenateName(column.name, name)); - res_columns[1]->insert(type->getName()); + res_columns[1]->insert(data.type->getName()); /// It's not trivial to calculate default expression for subcolumn. /// So, leave it empty. @@ -150,7 +150,7 @@ BlockIO InterpreterDescribeQuery::execute() res_columns[6]->insertDefault(); res_columns[7]->insert(1u); - }); + }, column.type->getDefaultSerialization(), column.type, nullptr); } } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index e334cd486ef..48fd9e583bf 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -35,8 +35,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; - }, - {}); + }); } NameSet remove_files; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 11e080fda6c..2f25cf7d12a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -98,7 +98,7 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( auto mrk_checksum = checksums.files.find(file_name + index_granularity_info.marks_file_extension); if (mrk_checksum != checksums.files.end()) size.marks += mrk_checksum->second.file_size; - }, {}); + }); return size; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 23a7b205a1b..5d17d6235e1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -39,19 +39,21 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc) { - IDataType::StreamCallbackWithType callback = [&] (const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) + ISerialization::StreamCallback callback = [&](const auto & substream_path) { + assert(!substream_path.empty()); String stream_name = ISerialization::getFileNameForStream(column, substream_path); /// Shared offsets for Nested type. if (compressed_streams.count(stream_name)) return; + const auto & subtype = substream_path.back().data.type; CompressionCodecPtr compression_codec; /// If we can use special codec than just get it if (ISerialization::isSpecialCompressionAllowed(substream_path)) - compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, subtype.get(), default_codec); else /// otherwise return only generic codecs and don't use info about data_type compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); @@ -63,7 +65,8 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, compressed_streams.emplace(stream_name, stream); }; - column.type->enumerateStreams(serializations[column.name], callback); + ISerialization::SubstreamPath path; + serializations[column.name]->enumerateStreams(path, callback, column.type, nullptr); } namespace diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 8fccfbb1f90..224a197c3c8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -90,17 +90,20 @@ void MergeTreeDataPartWriterWide::addStreams( const NameAndTypePair & column, const ASTPtr & effective_codec_desc) { - IDataType::StreamCallbackWithType callback = [&] (const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) + ISerialization::StreamCallback callback = [&](const auto & substream_path) { + assert(!substream_path.empty()); String stream_name = ISerialization::getFileNameForStream(column, substream_path); /// Shared offsets for Nested type. if (column_streams.count(stream_name)) return; + const auto & subtype = substream_path.back().data.type; CompressionCodecPtr compression_codec; + /// If we can use special codec then just get it if (ISerialization::isSpecialCompressionAllowed(substream_path)) - compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, subtype.get(), default_codec); else /// otherwise return only generic codecs and don't use info about the` data_type compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); @@ -113,7 +116,8 @@ void MergeTreeDataPartWriterWide::addStreams( settings.max_compress_block_size); }; - column.type->enumerateStreams(serializations[column.name], callback); + ISerialization::SubstreamPath path; + serializations[column.name]->enumerateStreams(path, callback, column.type, nullptr); } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index c898874f737..9b879283c10 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -220,7 +220,7 @@ void MergeTreeReaderCompact::readData( serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr); - auto subcolumn = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); + auto subcolumn = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), temp_column); /// TODO: Avoid extra copying. if (column->empty()) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b8941fc9d84..e7cfff86e5f 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -337,15 +337,14 @@ static NameToNameVector collectFilesForRenames( { /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. std::map stream_counts; - for (const NameAndTypePair & column : source_part->getColumns()) + for (const auto & column : source_part->getColumns()) { auto serialization = source_part->getSerializationForColumn(column); serialization->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; - }, - {}); + }); } NameToNameVector rename_vector; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8a234833da7..b6888dfddc6 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -141,8 +141,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin"; checksums_data.files[projection_file_name] = checksum_compressed_file(disk, projection_path + projection_file_name); - }, - {}); + }); } } @@ -219,7 +218,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { String file_name = ISerialization::getFileNameForStream(column, substream_path) + ".bin"; checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); - }, {}); + }); } } else diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..236972e6f2b 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -407,7 +407,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c storage.files[stream_name].data_file_path, columns.getCodecOrDefault(name_and_type.name), storage.max_compress_block_size); - }, settings.path); + }); settings.getter = createStreamGetter(name_and_type, written_streams); @@ -428,7 +428,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c mark.offset = stream_it->second.plain_offset + stream_it->second.plain->count(); out_marks.emplace_back(file.column_index, mark); - }, settings.path); + }); serialization->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); @@ -442,7 +442,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c if (streams.end() == it) throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR); it->second.compressed.next(); - }, settings.path); + }); } @@ -625,13 +625,12 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta * If this is a data type with multiple stream, get the first stream, that we assume have real row count. * (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays). */ - ISerialization::SubstreamPath substream_root_path; auto serialization = column.type->getDefaultSerialization(); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (filename.empty()) filename = ISerialization::getFileNameForStream(column, substream_path); - }, substream_root_path); + }); Files::const_iterator it = files.find(filename); if (files.end() == it) @@ -745,9 +744,8 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const size.data_compressed += file_sizes[fileName(it->second.data_file_path)]; }; - ISerialization::SubstreamPath substream_path; auto serialization = column.type->getDefaultSerialization(); - serialization->enumerateStreams(stream_callback, substream_path); + serialization->enumerateStreams(stream_callback); } return column_sizes; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index d1778342ec5..f899f8e1da6 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -448,9 +448,8 @@ void StorageTinyLog::addFiles(const NameAndTypePair & column) } }; - ISerialization::SubstreamPath substream_path; auto serialization = type->getDefaultSerialization(); - serialization->enumerateStreams(stream_callback, substream_path); + serialization->enumerateStreams(stream_callback); } @@ -544,9 +543,8 @@ IStorage::ColumnSizeByName StorageTinyLog::getColumnSizes() const size.data_compressed += file_sizes[fileName(it->second.data_file_path)]; }; - ISerialization::SubstreamPath substream_path; auto serialization = column.type->getDefaultSerialization(); - serialization->enumerateStreams(stream_callback, substream_path); + serialization->enumerateStreams(stream_callback); } return column_sizes; From 386d47cb227b580a63635e1b94a9d8a765d5cb97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Oct 2021 10:49:12 +0300 Subject: [PATCH 625/950] Fix typo --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index 341730bd82d..435bac85bc4 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -4,4 +4,4 @@ SYSTEM FLUSH LOGS; -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and databaser=currentDatabase() and type = 'QueryFinish'; +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; From 111c0672be7b99156218ad7a330420fb960a2d38 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Oct 2021 11:58:33 +0300 Subject: [PATCH 626/950] Start server under gdb in functional tests --- docker/test/stateless/run.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ce1d1b59a55..ec0af024b8b 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,6 +45,23 @@ else sudo clickhouse start fi +echo " +set follow-fork-mode child +handle all noprint +handle SIGSEGV stop print +handle SIGBUS stop print +handle SIGABRT stop print +continue +thread apply all backtrace +detach +quit +" > script.gdb + +# FIXME Hung check may work incorrectly because of attached gdb +# 1. False positives are possible +# 2. We cannot attach another gdb to get stacktraces if some queries hung +gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From baf75065e3614a71a28a837df4ac03b345dd54df Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 12:34:52 +0300 Subject: [PATCH 627/950] Update 02024_create_dictionary_with_comment.sql --- .../0_stateless/02024_create_dictionary_with_comment.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql index bbe2fa7066b..00557e4cf5c 100644 --- a/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql +++ b/tests/queries/0_stateless/02024_create_dictionary_with_comment.sql @@ -30,4 +30,4 @@ COMMENT 'Test dictionary with comment'; SHOW CREATE DICTIONARY 2024_dictionary_with_comment; SELECT comment FROM system.dictionaries WHERE name == '2024_dictionary_with_comment' AND database == currentDatabase(); -DROP DICTIONARY IF EXISTS 2024_dictionary_with_comment; \ No newline at end of file +DROP DICTIONARY IF EXISTS 2024_dictionary_with_comment; From 01ac2fea7991955ad68c8c0a4304fb0649ea84f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Oct 2021 12:44:41 +0300 Subject: [PATCH 628/950] Update 00167_read_bytes_from_fs.sql --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index 435bac85bc4..ee3e6b94537 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -4,4 +4,4 @@ SYSTEM FLUSH LOGS; -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM datasets.hits_v1 SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; From 562c8984a49068d406aae7e405d435fb08bc3bae Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 13:15:45 +0300 Subject: [PATCH 629/950] Added documentation --- .../functions/string-functions.md | 28 +++++++++++++++++-- src/Functions/tokenExtractors.cpp | 2 +- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c7c84c5aca6..4e70002c880 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -28,7 +28,7 @@ The function also works for [arrays](array-functions.md#function-empty) or [UUID **Returned value** -- Returns `1` for an empty string or `0` for a non-empty string. +- Returns `1` for an empty string or `0` for a non-empty string. Type: [UInt8](../data-types/int-uint.md). @@ -68,7 +68,7 @@ The function also works for [arrays](array-functions.md#function-notempty) or [U **Returned value** -- Returns `1` for a non-empty string or `0` for an empty string string. +- Returns `1` for a non-empty string or `0` for an empty string string. Type: [UInt8](../data-types/int-uint.md). @@ -313,6 +313,30 @@ SELECT toValidUTF8('\x61\xF0\x80\x80\x80b'); └───────────────────────┘ ``` +## tokens {#tokens} + +Split string into tokens using non-alpha numeric ASCII characters as separators. + +**Arguments** + +- `input_string` — Any set of bytes represented as the [String](../../sql-reference/data-types/string.md) data type object. + +**Returned value** + +- The resulting array of tokens from input string. + +Type: [Array](../data-types/array.md). + +**Example** + +``` +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens +``` + ## repeat {#repeat} Repeats a string as many times as specified and concatenates the replicated values as a single string. diff --git a/src/Functions/tokenExtractors.cpp b/src/Functions/tokenExtractors.cpp index f15cfb4cb62..be232c3b742 100644 --- a/src/Functions/tokenExtractors.cpp +++ b/src/Functions/tokenExtractors.cpp @@ -53,7 +53,7 @@ public: auto ngram_input_argument_type = WhichDataType(arguments[0].type); if (!ngram_input_argument_type.isStringOrFixedString()) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} second argument type should be String or FixedString. Actual {}", + "Function {} first argument type should be String or FixedString. Actual {}", getName(), arguments[0].type->getName()); From b8547262280362388ea492736f71fd464cd821ba Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 13:19:53 +0300 Subject: [PATCH 630/950] Updated documentation --- docs/en/sql-reference/functions/string-functions.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 4e70002c880..05c291bb41d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -329,12 +329,14 @@ Type: [Array](../data-types/array.md). **Example** -``` +``` sql SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; ``` ``` text -SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ ``` ## repeat {#repeat} From 96bee8d241c5dfa605c984f69c9f8ca25c40fbdd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 13:20:52 +0300 Subject: [PATCH 631/950] Fixed test --- tests/queries/0_stateless/02028_tokens.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02028_tokens.sql b/tests/queries/0_stateless/02028_tokens.sql index bbb6c8b79ec..16c88397cba 100644 --- a/tests/queries/0_stateless/02028_tokens.sql +++ b/tests/queries/0_stateless/02028_tokens.sql @@ -14,4 +14,4 @@ SELECT tokens(materialize('test1,;\ test2,;\ test3,;\ test4')); SELECT tokens(materialize('ё ё జ్ఞ‌ా 本気ですか ﷺ ᾂ ΐ שּ')); SELECT tokens(materialize('ё, ё, జ్ఞ‌ా, 本気ですか, ﷺ, ᾂ, ΐ, שּ')); SELECT tokens(materialize('ё, ё, జ్ఞ‌ా, 本気ですか, ﷺ, ᾂ, ΐ, שּ')); -SELECT tokens(materialize('ё;\ ё;\ జ్ఞ‌ా;\ 本気ですか;\ ﷺ;\ ᾂ;\ ΐ;\ שּ')); \ No newline at end of file +SELECT tokens(materialize('ё;\ ё;\ జ్ఞ‌ా;\ 本気ですか;\ ﷺ;\ ᾂ;\ ΐ;\ שּ')); From 90ff7f05fd8e85184591d24be67c125165f4f18b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Oct 2021 13:21:41 +0300 Subject: [PATCH 632/950] Start keeper asynchronously if has connection to other nodes --- programs/keeper/Keeper.cpp | 2 +- programs/server/Server.cpp | 5 +- src/Coordination/CoordinationSettings.h | 2 +- src/Coordination/KeeperDispatcher.cpp | 16 +++- src/Coordination/KeeperDispatcher.h | 7 +- src/Coordination/KeeperServer.cpp | 1 + src/Coordination/KeeperServer.h | 6 ++ src/Interpreters/Context.cpp | 23 ++++- src/Interpreters/Context.h | 7 +- src/Server/KeeperTCPHandler.cpp | 5 +- tests/integration/helpers/cluster.py | 3 + .../test_keeper_three_nodes_start/__init__.py | 1 + .../configs/enable_keeper1.xml | 32 +++++++ .../configs/enable_keeper2.xml | 32 +++++++ .../test_keeper_three_nodes_start/test.py | 32 +++++++ .../__init__.py | 1 + .../configs/enable_keeper1.xml | 32 +++++++ .../configs/enable_keeper2.xml | 32 +++++++ .../configs/enable_keeper3.xml | 32 +++++++ .../configs/keeper_conf.xml | 16 ++++ .../test_keeper_three_nodes_two_alive/test.py | 84 +++++++++++++++++++ 21 files changed, 357 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_keeper_three_nodes_start/__init__.py create mode 100644 tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_three_nodes_start/test.py create mode 100644 tests/integration/test_keeper_three_nodes_two_alive/__init__.py create mode 100644 tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml create mode 100644 tests/integration/test_keeper_three_nodes_two_alive/configs/keeper_conf.xml create mode 100644 tests/integration/test_keeper_three_nodes_two_alive/test.py diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 28bbb95e01d..a3034150219 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -359,7 +359,7 @@ int Keeper::main(const std::vector & /*args*/) auto servers = std::make_shared>(); /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. - global_context->initializeKeeperDispatcher(); + global_context->initializeKeeperDispatcher(/* start_async = */false); for (const auto & listen_host : listen_hosts) { /// TCP Keeper diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 01033570926..c2c53150daf 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -986,8 +986,9 @@ if (ThreadFuzzer::instance().isEffective()) if (config().has("keeper_server")) { #if USE_NURAFT - /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. - global_context->initializeKeeperDispatcher(); + bool has_connection = has_zookeeper && global_context->tryCheckZooKeeperConnection(); + /// Initialize keeper RAFT. + global_context->initializeKeeperDispatcher(has_connection); for (const auto & listen_host : listen_hosts) { /// TCP Keeper diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 805dedab89c..00d443822e6 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -28,7 +28,7 @@ struct Settings; M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \ - M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \ + M(Milliseconds, startup_timeout, 180000, "How many time we will until RAFT to start", 0) \ M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ M(UInt64, rotate_log_storage_interval, 100000, "How many records will be stored in one log storage file", 0) \ M(UInt64, snapshots_to_keep, 3, "How many compressed snapshots to keep on disk", 0) \ diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index b4dc367ff62..8d8f98e175e 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -241,7 +241,7 @@ bool KeeperDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & requ return true; } -void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper) +void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async) { LOG_DEBUG(log, "Initializing storage dispatcher"); int myid = config.getInt("keeper_server.server_id"); @@ -262,8 +262,16 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf server->startup(); LOG_DEBUG(log, "Server initialized, waiting for quorum"); - server->waitInit(); - LOG_DEBUG(log, "Quorum initialized"); + if (!start_async) + { + server->waitInit(); + LOG_DEBUG(log, "Quorum initialized"); + } + else + { + LOG_INFO(log, "Starting Keeper asynchronously, server will accept connections to Keeper when it will be ready"); + } + } catch (...) { @@ -363,7 +371,7 @@ void KeeperDispatcher::sessionCleanerTask() try { /// Only leader node must check dead sessions - if (isLeader()) + if (server->checkInit() && isLeader()) { auto dead_sessions = server->getDeadSessions(); diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index f49063f8dea..8f19b081e26 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -100,7 +100,12 @@ public: /// Initialization from config. /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) - void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper); + void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper, bool start_async); + + bool checkInit() const + { + return server && server->checkInit(); + } /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b27170c8ba1..56d28058991 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -353,6 +353,7 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ void KeeperServer::waitInit() { std::unique_lock lock(initialized_mutex); + int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds(); if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag.load(); })) throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization"); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index a7e96156dc1..8e10d053471 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -80,6 +80,12 @@ public: /// Wait server initialization (see callbackFunc) void waitInit(); + /// Return true if KeeperServer initialized + bool checkInit() const + { + return initialized_flag; + } + void shutdown(); int getServerID() const { return server_id; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e4c6de8853b..4ffd69af35b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1749,6 +1749,20 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } + +bool Context::tryCheckZooKeeperConnection() const +{ + try + { + getZooKeeper(); + return true; + } + catch (...) + { + return false; + } +} + UInt32 Context::getZooKeeperSessionUptime() const { std::lock_guard lock(shared->zookeeper_mutex); @@ -1776,19 +1790,24 @@ void Context::setSystemZooKeeperLogAfterInitializationIfNeeded() zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log); } -void Context::initializeKeeperDispatcher() const +void Context::initializeKeeperDispatcher(bool start_async) const { #if USE_NURAFT std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); + if (shared->keeper_storage_dispatcher) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); const auto & config = getConfigRef(); if (config.has("keeper_server")) { + bool is_standalone_app = getApplicationType() == ApplicationType::KEEPER; + if (start_async && !is_standalone_app) + LOG_INFO(shared->log, "Connected to ZooKeeper (or Keeper) before internal Keeper start, will wait for Keeper asynchronously"); + shared->keeper_storage_dispatcher = std::make_shared(); - shared->keeper_storage_dispatcher->initialize(config, getApplicationType() == ApplicationType::KEEPER); + shared->keeper_storage_dispatcher->initialize(config, is_standalone_app, start_async); } #endif } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f6cded2b1d1..07c90049bef 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -666,12 +666,17 @@ public: /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; + /// Try to connect to ZooKeeper using getZooKeeper. Useful for internal + /// Keeper start (check connection to some other node). Return true if + /// connected successfully (without exception). + bool tryCheckZooKeeperConnection() const; + UInt32 getZooKeeperSessionUptime() const; #if USE_NURAFT std::shared_ptr & getKeeperDispatcher() const; #endif - void initializeKeeperDispatcher() const; + void initializeKeeperDispatcher(bool start_async) const; void shutdownKeeperDispatcher() const; /// Set auxiliary zookeepers configuration at server starting or configuration reloading. diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 7ebbda9dfe6..88b7e68127e 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -286,7 +286,7 @@ void KeeperTCPHandler::runImpl() return; } - if (keeper_dispatcher->hasLeader()) + if (keeper_dispatcher->checkInit() && keeper_dispatcher->hasLeader()) { try { @@ -306,7 +306,8 @@ void KeeperTCPHandler::runImpl() } else { - LOG_WARNING(log, "Ignoring user request, because no alive leader exist"); + String reason = keeper_dispatcher->checkInit() ? "server is not initialized yet" : "no alive leader exists"; + LOG_WARNING(log, "Ignoring user request, because {}", reason); sendHandshake(false); return; } diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 51b7bfcbcb8..3854cadaba5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2320,6 +2320,9 @@ class ClickHouseInstance: def replace_config(self, path_to_config, replacement): self.exec_in_container(["bash", "-c", "echo '{}' > {}".format(replacement, path_to_config)]) + def replace_in_config(self, path_to_config, replace, replacement): + self.exec_in_container(["bash", "-c", f"sed -i 's/{replace}/{replacement}/g' {path_to_config}"]) + def create_dir(self, destroy_dir=True): """Create the instance directory and all the needed files there.""" diff --git a/tests/integration/test_keeper_three_nodes_start/__init__.py b/tests/integration/test_keeper_three_nodes_start/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_start/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml new file mode 100644 index 00000000000..bc62d817074 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper1.xml @@ -0,0 +1,32 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + non_existing_node + 44444 + + + + diff --git a/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml new file mode 100644 index 00000000000..a6c476fb449 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_start/configs/enable_keeper2.xml @@ -0,0 +1,32 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + non_existing_node + 44444 + + + + diff --git a/tests/integration/test_keeper_three_nodes_start/test.py b/tests/integration/test_keeper_three_nodes_start/test.py new file mode 100644 index 00000000000..7828f21d0d7 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_start/test.py @@ -0,0 +1,32 @@ +#!/usr/bin/env python3 + +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml'], stay_alive=True) + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + +def test_smoke(): + try: + cluster.start() + + node1_zk = get_fake_zk("node1") + node1_zk.create("/test_alive", b"aaaa") + + finally: + cluster.shutdown() diff --git a/tests/integration/test_keeper_three_nodes_two_alive/__init__.py b/tests/integration/test_keeper_three_nodes_two_alive/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_two_alive/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml new file mode 100644 index 00000000000..510424715c4 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper1.xml @@ -0,0 +1,32 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml new file mode 100644 index 00000000000..264601d8c98 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper2.xml @@ -0,0 +1,32 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml new file mode 100644 index 00000000000..7f9775939bb --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/enable_keeper3.xml @@ -0,0 +1,32 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + diff --git a/tests/integration/test_keeper_three_nodes_two_alive/configs/keeper_conf.xml b/tests/integration/test_keeper_three_nodes_two_alive/configs/keeper_conf.xml new file mode 100644 index 00000000000..384e984f210 --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_two_alive/configs/keeper_conf.xml @@ -0,0 +1,16 @@ + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + diff --git a/tests/integration/test_keeper_three_nodes_two_alive/test.py b/tests/integration/test_keeper_three_nodes_two_alive/test.py new file mode 100644 index 00000000000..2c13d3ef22b --- /dev/null +++ b/tests/integration/test_keeper_three_nodes_two_alive/test.py @@ -0,0 +1,84 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml', 'configs/keeper_conf.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml', 'configs/keeper_conf.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml', 'configs/keeper_conf.xml'], stay_alive=True) + + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def start(node): + node.start_clickhouse() + + +def test_start_offline(started_cluster): + p = Pool(3) + try: + node1_zk = get_fake_zk("node1") + node1_zk.create("/test_alive", b"aaaa") + + node1.stop_clickhouse() + node2.stop_clickhouse() + node3.stop_clickhouse() + + time.sleep(3) + p.map(start, [node2, node3]) + + node2_zk = get_fake_zk("node2") + node2_zk.create("/test_dead", b"data") + finally: + p.map(start, [node1, node2, node3]) + + +def test_start_non_existing(started_cluster): + p = Pool(3) + try: + node1.stop_clickhouse() + node2.stop_clickhouse() + node3.stop_clickhouse() + + node1.replace_in_config('/etc/clickhouse-server/config.d/enable_keeper1.xml', 'node3', 'non_existing_node') + node2.replace_in_config('/etc/clickhouse-server/config.d/enable_keeper2.xml', 'node3', 'non_existing_node') + + time.sleep(3) + p.map(start, [node2, node1]) + + node2_zk = get_fake_zk("node2") + node2_zk.create("/test_non_exising", b"data") + finally: + node1.replace_in_config('/etc/clickhouse-server/config.d/enable_keeper1.xml', 'non_existing_node', 'node3') + node2.replace_in_config('/etc/clickhouse-server/config.d/enable_keeper2.xml', 'non_existing_node', 'node3') + p.map(start, [node1, node2, node3]) + +def test_restart_third_node(started_cluster): + node1_zk = get_fake_zk("node1") + node1_zk.create("/test_restart", b"aaaa") + + node3.restart_clickhouse() + + assert node3.contains_in_log("Connected to ZooKeeper (or Keeper) before internal Keeper start") From 664e80af5adf396cac63e4b06b3c7b081f598c3f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Oct 2021 13:23:39 +0300 Subject: [PATCH 633/950] Update run.sh --- docker/test/stateless/run.sh | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index ec0af024b8b..ce1d1b59a55 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,23 +45,6 @@ else sudo clickhouse start fi -echo " -set follow-fork-mode child -handle all noprint -handle SIGSEGV stop print -handle SIGBUS stop print -handle SIGABRT stop print -continue -thread apply all backtrace -detach -quit -" > script.gdb - -# FIXME Hung check may work incorrectly because of attached gdb -# 1. False positives are possible -# 2. We cannot attach another gdb to get stacktraces if some queries hung -gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & - if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ From ab28c6c855dfc57b40938f858a99c091896d166e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 13:25:43 +0300 Subject: [PATCH 634/950] Remove BlockInputStream interfaces. --- programs/copier/Internals.cpp | 8 - programs/copier/Internals.h | 2 - programs/library-bridge/Handlers.cpp | 17 +- .../library-bridge/SharedLibraryHandler.h | 1 - programs/odbc-bridge/MainHandler.cpp | 1 - src/Client/IServerConnection.h | 1 - src/Core/ExternalTable.cpp | 1 - src/DataStreams/BlockStreamProfileInfo.cpp | 75 ---- src/DataStreams/BlockStreamProfileInfo.h | 9 - src/DataStreams/IBlockInputStream.cpp | 359 ------------------ src/DataStreams/IBlockInputStream.h | 271 ------------- src/DataStreams/IBlockOutputStream.h | 70 ---- src/DataStreams/IBlockStream_fwd.h | 17 - src/DataStreams/InternalTextLogs.h | 2 +- src/DataStreams/NativeReader.h | 2 +- src/DataStreams/TemporaryFileStream.cpp | 2 - src/DataStreams/TemporaryFileStream.h | 1 - src/DataStreams/copyData.cpp | 86 ----- src/DataStreams/copyData.h | 27 -- .../MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Dictionaries/DictionarySourceHelpers.cpp | 1 - src/Dictionaries/HTTPDictionarySource.cpp | 1 - src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatFactory.h | 1 - src/Interpreters/Aggregator.h | 1 - src/Interpreters/Context.h | 1 - src/Interpreters/ExpressionAnalyzer.cpp | 1 - src/Interpreters/ExpressionAnalyzer.h | 1 - src/Interpreters/InterpreterExistsQuery.cpp | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 1 - src/Interpreters/InterpreterInsertQuery.h | 1 - src/Interpreters/InterpreterSelectQuery.h | 8 - .../InterpreterShowCreateQuery.cpp | 1 - src/Interpreters/InterpreterWatchQuery.cpp | 1 - src/Interpreters/InterpreterWatchQuery.h | 2 - src/Interpreters/JoinSwitcher.h | 1 - src/Interpreters/ProcessList.cpp | 1 - src/Interpreters/SortedBlocksWriter.h | 1 - src/Interpreters/TableJoin.h | 1 - src/Interpreters/executeDDLQueryOnCluster.h | 1 - src/Interpreters/executeQuery.cpp | 2 - .../Formats/OutputStreamToOutputFormat.cpp | 43 --- .../Formats/OutputStreamToOutputFormat.h | 39 -- src/Processors/QueryPipelineBuilder.h | 1 - .../Transforms/CreatingSetsTransform.cpp | 1 - .../Transforms/CreatingSetsTransform.h | 1 - .../Transforms/buildPushingToViewsChain.h | 1 - src/Server/MySQLHandler.cpp | 1 - src/Server/TCPHandler.h | 1 - src/Storages/IStorage.h | 1 - src/Storages/Kafka/StorageKafka.cpp | 1 - src/Storages/LiveView/StorageLiveView.cpp | 1 - .../MergeTree/IMergeTreeDataPartWriter.h | 1 - src/Storages/MergeTree/MergeTreeData.cpp | 1 - .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 - .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 1 - .../PostgreSQLReplicationHandler.cpp | 1 - src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 1 - src/Storages/StorageBuffer.h | 1 - src/Storages/StorageExecutable.cpp | 1 - src/Storages/StorageFile.cpp | 3 - src/Storages/StorageLog.cpp | 2 - src/Storages/StorageMaterializedView.cpp | 1 - src/Storages/StorageMemory.h | 1 - src/Storages/StorageMongoDB.cpp | 1 - src/Storages/StorageMySQL.cpp | 1 - src/Storages/StoragePostgreSQL.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/StorageS3.cpp | 1 - src/Storages/StorageS3Cluster.cpp | 1 - src/Storages/StorageSet.cpp | 1 + src/Storages/StorageStripeLog.cpp | 1 - src/Storages/StorageURL.cpp | 1 - src/Storages/StorageXDBC.cpp | 1 - .../System/StorageSystemZooKeeper.cpp | 1 - src/Storages/tests/gtest_storage_log.cpp | 1 - src/TableFunctions/ITableFunctionFileLike.cpp | 1 - src/TableFunctions/TableFunctionExecutable.h | 1 - 79 files changed, 15 insertions(+), 1090 deletions(-) delete mode 100644 src/DataStreams/IBlockInputStream.cpp delete mode 100644 src/DataStreams/IBlockInputStream.h delete mode 100644 src/DataStreams/IBlockOutputStream.h delete mode 100644 src/DataStreams/IBlockStream_fwd.h delete mode 100644 src/DataStreams/copyData.cpp delete mode 100644 src/DataStreams/copyData.h delete mode 100644 src/Processors/Formats/OutputStreamToOutputFormat.cpp delete mode 100644 src/Processors/Formats/OutputStreamToOutputFormat.h diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index c5e702cd1dc..84283777c8f 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -57,14 +57,6 @@ std::shared_ptr createASTStorageDistributed( } -BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) -{ - return std::make_shared( - stream, - std::numeric_limits::max(), - std::numeric_limits::max()); -} - Block getBlockWithAllStreamData(QueryPipeline pipeline) { QueryPipelineBuilder builder; diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 9e648060592..a9d8ca726fe 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -50,8 +50,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp index 2f6dca5ee65..abc5118baad 100644 --- a/programs/library-bridge/Handlers.cpp +++ b/programs/library-bridge/Handlers.cpp @@ -1,7 +1,6 @@ #include "Handlers.h" #include "SharedLibraryHandlerFactory.h" -#include #include #include #include @@ -10,11 +9,13 @@ #include #include #include -#include #include +#include #include #include +#include #include +#include #include #include @@ -189,8 +190,10 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe ReadBufferFromString read_block_buf(params.get("null_values")); auto format = getContext()->getInputFormat(FORMAT, read_block_buf, *sample_block, DEFAULT_BLOCK_SIZE); - auto reader = std::make_shared(format); - auto sample_block_with_nulls = reader->read(); + QueryPipeline pipeline(Pipe(std::move(format))); + PullingPipelineExecutor executor(pipeline); + Block sample_block_with_nulls; + executor.pull(sample_block_with_nulls); LOG_DEBUG(log, "Dictionary sample block with null values: {}", sample_block_with_nulls.dumpStructure()); @@ -281,8 +284,10 @@ void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServe auto & read_buf = request.getStream(); auto format = getContext()->getInputFormat(FORMAT, read_buf, *requested_sample_block, DEFAULT_BLOCK_SIZE); - auto reader = std::make_shared(format); - auto block = reader->read(); + QueryPipeline pipeline(std::move(format)); + PullingPipelineExecutor executor(pipeline); + Block block; + executor.pull(block); auto library_handler = SharedLibraryHandlerFactory::instance().get(dictionary_id); if (!library_handler) diff --git a/programs/library-bridge/SharedLibraryHandler.h b/programs/library-bridge/SharedLibraryHandler.h index f9d2fe43cb2..de1d098dc8d 100644 --- a/programs/library-bridge/SharedLibraryHandler.h +++ b/programs/library-bridge/SharedLibraryHandler.h @@ -2,7 +2,6 @@ #include #include -#include #include "LibraryUtils.h" diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 51abe207095..6a2e967d179 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -4,7 +4,6 @@ #include "ODBCBlockInputStream.h" #include "ODBCBlockOutputStream.h" #include "getIdentifierQuote.h" -#include #include #include #include diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 5584cf72bbf..42886c72182 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -6,7 +6,6 @@ #include #include -#include #include #include diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 7619d60d84e..4dd8b0cf016 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/DataStreams/BlockStreamProfileInfo.cpp b/src/DataStreams/BlockStreamProfileInfo.cpp index 09ad8a8e4ac..9a06d905223 100644 --- a/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/src/DataStreams/BlockStreamProfileInfo.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -47,16 +46,12 @@ void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool sk size_t BlockStreamProfileInfo::getRowsBeforeLimit() const { - if (!calculated_rows_before_limit) - calculateRowsBeforeLimit(); return rows_before_limit; } bool BlockStreamProfileInfo::hasAppliedLimit() const { - if (!calculated_rows_before_limit) - calculateRowsBeforeLimit(); return applied_limit; } @@ -73,74 +68,4 @@ void BlockStreamProfileInfo::update(size_t num_rows, size_t num_bytes) bytes += num_bytes; } - -void BlockStreamProfileInfo::collectInfosForStreamsWithName(const char * name, BlockStreamProfileInfos & res) const -{ - if (!parent) - return; - - if (parent->getName() == name) - { - res.push_back(this); - return; - } - - parent->forEachChild([&] (IBlockInputStream & child) - { - child.getProfileInfo().collectInfosForStreamsWithName(name, res); - return false; - }); -} - - -void BlockStreamProfileInfo::calculateRowsBeforeLimit() const -{ - calculated_rows_before_limit = true; - - /// is there a Limit? - BlockStreamProfileInfos limits; - collectInfosForStreamsWithName("Limit", limits); - - if (!limits.empty()) - { - applied_limit = true; - - /** Take the number of lines read below `PartialSorting`, if any, or below `Limit`. - * This is necessary, because sorting can return only part of the rows. - */ - BlockStreamProfileInfos partial_sortings; - collectInfosForStreamsWithName("PartialSorting", partial_sortings); - - BlockStreamProfileInfos & limits_or_sortings = partial_sortings.empty() ? limits : partial_sortings; - - for (const BlockStreamProfileInfo * info_limit_or_sort : limits_or_sortings) - { - info_limit_or_sort->parent->forEachChild([&] (IBlockInputStream & child) - { - rows_before_limit += child.getProfileInfo().rows; - return false; - }); - } - } - else - { - /// Then the data about `rows_before_limit` can be in `RemoteBlockInputStream` (come from a remote server). - BlockStreamProfileInfos remotes; - collectInfosForStreamsWithName("Remote", remotes); - collectInfosForStreamsWithName("TreeExecutor", remotes); - - if (remotes.empty()) - return; - - for (const auto & info : remotes) - { - if (info->applied_limit) - { - applied_limit = true; - rows_before_limit += info->rows_before_limit; - } - } - } -} - } diff --git a/src/DataStreams/BlockStreamProfileInfo.h b/src/DataStreams/BlockStreamProfileInfo.h index 688bdfc91fc..1707b941445 100644 --- a/src/DataStreams/BlockStreamProfileInfo.h +++ b/src/DataStreams/BlockStreamProfileInfo.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -16,9 +15,6 @@ class WriteBuffer; /// Information for profiling. See IBlockInputStream.h struct BlockStreamProfileInfo { - /// Info about stream object this profile info refers to. - IBlockInputStream * parent = nullptr; - bool started = false; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Time with waiting time @@ -28,9 +24,6 @@ struct BlockStreamProfileInfo using BlockStreamProfileInfos = std::vector; - /// Collect BlockStreamProfileInfo for the nearest sources in the tree named `name`. Example; collect all info for PartialSorting streams. - void collectInfosForStreamsWithName(const char * name, BlockStreamProfileInfos & res) const; - /** Get the number of rows if there were no LIMIT. * If there is no LIMIT, 0 is returned. * If the query does not contain ORDER BY, the number can be underestimated - return the number of rows in blocks that were read before LIMIT reached. @@ -59,8 +52,6 @@ struct BlockStreamProfileInfo } private: - void calculateRowsBeforeLimit() const; - /// For these fields we make accessors, because they must be calculated beforehand. mutable bool applied_limit = false; /// Whether LIMIT was applied mutable size_t rows_before_limit = 0; diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp deleted file mode 100644 index e57d6903673..00000000000 --- a/src/DataStreams/IBlockInputStream.cpp +++ /dev/null @@ -1,359 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; - extern const Event SelectedRows; - extern const Event SelectedBytes; -} - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int QUERY_WAS_CANCELLED; - extern const int TOO_MANY_ROWS; - extern const int TOO_MANY_BYTES; - extern const int TOO_MANY_ROWS_OR_BYTES; - extern const int LOGICAL_ERROR; -} - - -/// It's safe to access children without mutex as long as these methods are called before first call to `read()` or `readPrefix()`. - - -Block IBlockInputStream::read() -{ - if (total_rows_approx) - { - progressImpl(Progress(0, 0, total_rows_approx)); - total_rows_approx = 0; - } - - if (!info.started) - { - info.total_stopwatch.start(); - info.started = true; - } - - Block res; - - if (isCancelledOrThrowIfKilled()) - return res; - - if (!checkTimeLimit()) - limit_exceeded_need_break = true; - - if (!limit_exceeded_need_break) - res = readImpl(); - - if (res) - { - info.update(res); - - if (enabled_extremes) - updateExtremes(res); - - if (limits.mode == LimitsMode::LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) - limit_exceeded_need_break = true; - - if (quota) - checkQuota(res); - } - else - { - /** If the stream is over, then we will ask all children to abort the execution. - * This makes sense when running a query with LIMIT - * - there is a situation when all the necessary data has already been read, - * but children sources are still working, - * herewith they can work in separate threads or even remotely. - */ - cancel(false); - } - - progress(Progress(res.rows(), res.bytes())); - -#ifndef NDEBUG - if (res) - { - Block header = getHeader(); - if (header) - assertBlocksHaveEqualStructure(res, header, getName()); - } -#endif - - return res; -} - - -void IBlockInputStream::readPrefix() -{ -#ifndef NDEBUG - if (!read_prefix_is_called) - read_prefix_is_called = true; - else - throw Exception("readPrefix is called twice for " + getName() + " stream", ErrorCodes::LOGICAL_ERROR); -#endif - - readPrefixImpl(); - - forEachChild([&] (IBlockInputStream & child) - { - child.readPrefix(); - return false; - }); -} - - -void IBlockInputStream::readSuffix() -{ -#ifndef NDEBUG - if (!read_suffix_is_called) - read_suffix_is_called = true; - else - throw Exception("readSuffix is called twice for " + getName() + " stream", ErrorCodes::LOGICAL_ERROR); -#endif - - forEachChild([&] (IBlockInputStream & child) - { - child.readSuffix(); - return false; - }); - - readSuffixImpl(); -} - - -void IBlockInputStream::updateExtremes(Block & block) -{ - size_t num_columns = block.columns(); - - if (!extremes) - { - MutableColumns extremes_columns(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnPtr & src = block.safeGetByPosition(i).column; - - if (isColumnConst(*src)) - { - /// Equal min and max. - extremes_columns[i] = src->cloneResized(2); - } - else - { - Field min_value; - Field max_value; - - src->getExtremes(min_value, max_value); - - extremes_columns[i] = src->cloneEmpty(); - - extremes_columns[i]->insert(min_value); - extremes_columns[i]->insert(max_value); - } - } - - extremes = block.cloneWithColumns(std::move(extremes_columns)); - } - else - { - for (size_t i = 0; i < num_columns; ++i) - { - ColumnPtr & old_extremes = extremes.safeGetByPosition(i).column; - - if (isColumnConst(*old_extremes)) - continue; - - Field min_value = (*old_extremes)[0]; - Field max_value = (*old_extremes)[1]; - - Field cur_min_value; - Field cur_max_value; - - block.safeGetByPosition(i).column->getExtremes(cur_min_value, cur_max_value); - - if (cur_min_value < min_value) - min_value = cur_min_value; - if (cur_max_value > max_value) - max_value = cur_max_value; - - MutableColumnPtr new_extremes = old_extremes->cloneEmpty(); - - new_extremes->insert(min_value); - new_extremes->insert(max_value); - - old_extremes = std::move(new_extremes); - } - } -} - - -bool IBlockInputStream::checkTimeLimit() const -{ - return limits.speed_limits.checkTimeLimit(info.total_stopwatch, limits.timeout_overflow_mode); -} - - -void IBlockInputStream::checkQuota(Block & block) -{ - switch (limits.mode) - { - case LimitsMode::LIMITS_TOTAL: - /// Checked in `progress` method. - break; - - case LimitsMode::LIMITS_CURRENT: - { - UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); - quota->used({Quota::RESULT_ROWS, block.rows()}, {Quota::RESULT_BYTES, block.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); - prev_elapsed = total_elapsed; - break; - } - } -} - - -void IBlockInputStream::progressImpl(const Progress & value) -{ - if (progress_callback) - progress_callback(value); - - if (process_list_elem) - { - if (!process_list_elem->updateProgressIn(value)) - cancel(/* kill */ true); - - /// The total amount of data processed or intended for processing in all leaf sources, possibly on remote servers. - - ProgressValues progress = process_list_elem->getProgressIn(); - size_t total_rows_estimate = std::max(progress.read_rows, progress.total_rows_to_read); - - /** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read. - * NOTE: Maybe it makes sense to have them checked directly in ProcessList? - */ - if (limits.mode == LimitsMode::LIMITS_TOTAL) - { - if (!limits.size_limits.check(total_rows_estimate, progress.read_bytes, "rows to read", - ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) - cancel(false); - } - - size_t total_rows = progress.total_rows_to_read; - - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); - - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) - { - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } - - limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); - - if (quota && limits.mode == LimitsMode::LIMITS_TOTAL) - quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes}); - } - - ProfileEvents::increment(ProfileEvents::SelectedRows, value.read_rows); - ProfileEvents::increment(ProfileEvents::SelectedBytes, value.read_bytes); -} - - -void IBlockInputStream::cancel(bool kill) -{ - if (kill) - is_killed = true; - - bool old_val = false; - if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) - return; - - forEachChild([&] (IBlockInputStream & child) - { - child.cancel(kill); - return false; - }); -} - - -bool IBlockInputStream::isCancelled() const -{ - return is_cancelled; -} - -bool IBlockInputStream::isCancelledOrThrowIfKilled() const -{ - if (!is_cancelled) - return false; - if (is_killed) - throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); - return true; -} - - -void IBlockInputStream::setProgressCallback(const ProgressCallback & callback) -{ - progress_callback = callback; - - forEachChild([&] (IBlockInputStream & child) - { - child.setProgressCallback(callback); - return false; - }); -} - - -void IBlockInputStream::setProcessListElement(QueryStatus * elem) -{ - process_list_elem = elem; - - forEachChild([&] (IBlockInputStream & child) - { - child.setProcessListElement(elem); - return false; - }); -} - - -Block IBlockInputStream::getTotals() -{ - if (totals) - return totals; - - Block res; - forEachChild([&] (IBlockInputStream & child) - { - res = child.getTotals(); - return bool(res); - }); - return res; -} - - -Block IBlockInputStream::getExtremes() -{ - if (extremes) - return extremes; - - Block res; - forEachChild([&] (IBlockInputStream & child) - { - res = child.getExtremes(); - return bool(res); - }); - return res; -} - -} diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h deleted file mode 100644 index 0e77ba81779..00000000000 --- a/src/DataStreams/IBlockInputStream.h +++ /dev/null @@ -1,271 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ -} - -class ProcessListElement; -class EnabledQuota; -class QueryStatus; - - -/** The stream interface for reading data by blocks from the database. - * Relational operations are supposed to be done also as implementations of this interface. - * Watches out at how the source of the blocks works. - * Lets you get information for profiling: rows per second, blocks per second, megabytes per second, etc. - * Allows you to stop reading data (in nested sources). - */ -class IBlockInputStream : public TypePromotion -{ - friend struct BlockStreamProfileInfo; - -public: - IBlockInputStream() { info.parent = this; } - virtual ~IBlockInputStream() = default; - - IBlockInputStream(const IBlockInputStream &) = delete; - IBlockInputStream & operator=(const IBlockInputStream &) = delete; - - /// To output the data stream transformation tree (query execution plan). - virtual String getName() const = 0; - - /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). - * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. - * It is guaranteed that method "read" returns blocks of exactly that structure. - */ - virtual Block getHeader() const = 0; - - virtual const BlockMissingValues & getMissingValues() const - { - static const BlockMissingValues none; - return none; - } - - /** Read next block. - * If there are no more blocks, return an empty block (for which operator `bool` returns false). - * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously. - * This also applies for readPrefix, readSuffix. - */ - Block read(); - - /** Read something before starting all data or after the end of all data. - * In the `readSuffix` function, you can implement a finalization that can lead to an exception. - * readPrefix() must be called before the first call to read(). - * readSuffix() should be called after read() returns an empty block, or after a call to cancel(), but not during read() execution. - */ - - /** The default implementation calls readPrefixImpl() on itself, and then readPrefix() recursively for all children. - * There are cases when you do not want `readPrefix` of children to be called synchronously, in this function, - * but you want them to be called, for example, in separate threads (for parallel initialization of children). - * Then overload `readPrefix` function. - */ - virtual void readPrefix(); - - /** The default implementation calls recursively readSuffix() on all children, and then readSuffixImpl() on itself. - * If this stream calls read() in children in a separate thread, this behavior is usually incorrect: - * readSuffix() of the child can not be called at the moment when the same child's read() is executed in another thread. - * In this case, you need to override this method so that readSuffix() in children is called, for example, after connecting streams. - */ - virtual void readSuffix(); - - /// Do not allow to change the table while the blocks stream and its children are alive. - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - - /// Get information about execution speed. - const BlockStreamProfileInfo & getProfileInfo() const { return info; } - - /** Get "total" values. - * The default implementation takes them from itself or from the first child source in which they are. - * The overridden method can perform some calculations. For example, apply an expression to the `totals` of the child source. - * There can be no total values - then an empty block is returned. - * - * Call this method only after all the data has been retrieved with `read`, - * otherwise there will be problems if any data at the same time is computed in another thread. - */ - virtual Block getTotals(); - - /// The same for minimums and maximums. - virtual Block getExtremes(); - - - /** Set the execution progress bar callback. - * The callback is passed to all child sources. - * By default, it is called for leaf sources, after each block. - * (But this can be overridden in the progress() method) - * The function takes the number of rows in the last block, the number of bytes in the last block. - * Note that the callback can be called from different threads. - */ - virtual void setProgressCallback(const ProgressCallback & callback); - - - /** In this method: - * - the progress callback is called; - * - the status of the query execution in ProcessList is updated; - * - checks restrictions and quotas that should be checked not within the same source, - * but over the total amount of resources spent in all sources at once (information in the ProcessList). - */ - virtual void progress(const Progress & value) - { - /// The data for progress is taken from leaf sources. - if (children.empty()) - progressImpl(value); - } - - void progressImpl(const Progress & value); - - - /** Set the pointer to the process list item. - * It is passed to all child sources. - * General information about the resources spent on the request will be written into it. - * Based on this information, the quota and some restrictions will be checked. - * This information will also be available in the SHOW PROCESSLIST request. - */ - virtual void setProcessListElement(QueryStatus * elem); - - /** Set the approximate total number of rows to read. - */ - void addTotalRowsApprox(size_t value) { total_rows_approx += value; } - - - /** Ask to abort the receipt of data as soon as possible. - * By default - just sets the flag is_cancelled and asks that all children be interrupted. - * This function can be called several times, including simultaneously from different threads. - * Have two modes: - * with kill = false only is_cancelled is set - streams will stop silently with returning some processed data. - * with kill = true also is_killed set - queries will stop with exception. - */ - virtual void cancel(bool kill); - - bool isCancelled() const; - bool isCancelledOrThrowIfKilled() const; - - /** Set limitations that checked on each block. */ - virtual void setLimits(const StreamLocalLimits & limits_) - { - limits = limits_; - } - - const StreamLocalLimits & getLimits() const - { - return limits; - } - - /** Set the quota. If you set a quota on the amount of raw data, - * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. - */ - virtual void setQuota(const std::shared_ptr & new_quota) - { - quota = new_quota; - } - - /// Enable calculation of minimums and maximums by the result columns. - void enableExtremes() { enabled_extremes = true; } - -protected: - /// Order is important: `table_locks` must be destroyed after `children` so that tables from - /// which child streams read are protected by the locks during the lifetime of the child streams. - std::vector table_locks; - - BlockInputStreams children; - std::shared_mutex children_mutex; - - BlockStreamProfileInfo info; - std::atomic is_cancelled{false}; - std::atomic is_killed{false}; - ProgressCallback progress_callback; - QueryStatus * process_list_elem = nullptr; - /// According to total_stopwatch in microseconds - UInt64 last_profile_events_update_time = 0; - - /// Additional information that can be generated during the work process. - - /// Total values during aggregation. - Block totals; - /// Minimums and maximums. The first row of the block - minimums, the second - the maximums. - Block extremes; - - - void addChild(const BlockInputStreamPtr & child) - { - std::unique_lock lock(children_mutex); - children.push_back(child); - } - - /** Check limits. - * But only those that can be checked within each separate stream. - */ - bool checkTimeLimit() const; - -#ifndef NDEBUG - bool read_prefix_is_called = false; - bool read_suffix_is_called = false; -#endif - -private: - bool enabled_extremes = false; - - /// The limit on the number of rows/bytes has been exceeded, and you need to stop execution on the next `read` call, as if the thread has run out. - bool limit_exceeded_need_break = false; - - /// Limitations and quotas. - - StreamLocalLimits limits; - - std::shared_ptr quota; /// If nullptr - the quota is not used. - UInt64 prev_elapsed = 0; - - /// The approximate total number of rows to read. For progress bar. - size_t total_rows_approx = 0; - - /// Derived classes must implement this function. - virtual Block readImpl() = 0; - - /// Here you can do a preliminary initialization. - virtual void readPrefixImpl() {} - - /// Here you need to do a finalization, which can lead to an exception. - virtual void readSuffixImpl() {} - - void updateExtremes(Block & block); - - /** Check quotas. - * But only those that can be checked within each separate stream. - */ - void checkQuota(Block & block); - - size_t checkDepthImpl(size_t max_depth, size_t level) const; - - template - void forEachChild(F && f) - { - /// NOTE: Acquire a read lock, therefore f() should be thread safe - std::shared_lock lock(children_mutex); - - // Reduce lock scope and avoid recursive locking since that is undefined for shared_mutex. - const auto children_copy = children; - lock.unlock(); - - for (auto & child : children_copy) - if (f(*child)) - return; - } - -}; - -} diff --git a/src/DataStreams/IBlockOutputStream.h b/src/DataStreams/IBlockOutputStream.h deleted file mode 100644 index 65ebd90769d..00000000000 --- a/src/DataStreams/IBlockOutputStream.h +++ /dev/null @@ -1,70 +0,0 @@ -#pragma once - -#include -#include -#include - -#include - -#include -#include -#include - - -namespace DB -{ - -struct Progress; - -/** Interface of stream for writing data (into table, filesystem, network, terminal, etc.) - */ -class IBlockOutputStream : private boost::noncopyable -{ -public: - IBlockOutputStream() = default; - - /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). - * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. - * You must pass blocks of exactly this structure to the 'write' method. - */ - virtual Block getHeader() const = 0; - - /** Write block. - */ - virtual void write(const Block & block) = 0; - - /** Write or do something before all data or after all data. - */ - virtual void writePrefix() {} - virtual void writeSuffix() {} - - /** Flush output buffers if any. - */ - virtual void flush() {} - - /** Methods to set additional information for output in formats, that support it. - */ - virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {} - virtual void setTotals(const Block & /*totals*/) {} - virtual void setExtremes(const Block & /*extremes*/) {} - - /** Notify about progress. Method could be called from different threads. - * Passed value are delta, that must be summarized. - */ - virtual void onProgress(const Progress & /*progress*/) {} - - /** Content-Type to set when sending HTTP response. - */ - virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; } - - virtual ~IBlockOutputStream() = default; - - /** Don't let to alter table while instance of stream is alive. - */ - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - -private: - std::vector table_locks; -}; - -} diff --git a/src/DataStreams/IBlockStream_fwd.h b/src/DataStreams/IBlockStream_fwd.h deleted file mode 100644 index d74a9528ed9..00000000000 --- a/src/DataStreams/IBlockStream_fwd.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class IBlockInputStream; -class IBlockOutputStream; - -using BlockInputStreamPtr = std::shared_ptr; -using BlockInputStreams = std::vector; -using BlockOutputStreamPtr = std::shared_ptr; -using BlockOutputStreams = std::vector; - -} diff --git a/src/DataStreams/InternalTextLogs.h b/src/DataStreams/InternalTextLogs.h index 1312c1d327c..a8b119b0f69 100644 --- a/src/DataStreams/InternalTextLogs.h +++ b/src/DataStreams/InternalTextLogs.h @@ -1,6 +1,6 @@ #pragma once -#include #include +#include namespace DB diff --git a/src/DataStreams/NativeReader.h b/src/DataStreams/NativeReader.h index cfd58bde2cc..95b03c71764 100644 --- a/src/DataStreams/NativeReader.h +++ b/src/DataStreams/NativeReader.h @@ -1,8 +1,8 @@ #pragma once -#include #include #include +#include namespace DB { diff --git a/src/DataStreams/TemporaryFileStream.cpp b/src/DataStreams/TemporaryFileStream.cpp index 826cf5508d8..4b7c9d50fe7 100644 --- a/src/DataStreams/TemporaryFileStream.cpp +++ b/src/DataStreams/TemporaryFileStream.cpp @@ -1,8 +1,6 @@ #include -#include #include #include -#include #include #include #include diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index c0c13605928..e288b5b30fa 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -4,7 +4,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/DataStreams/copyData.cpp b/src/DataStreams/copyData.cpp deleted file mode 100644 index a26052778a8..00000000000 --- a/src/DataStreams/copyData.cpp +++ /dev/null @@ -1,86 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -namespace -{ - -bool isAtomicSet(std::atomic * val) -{ - return ((val != nullptr) && val->load(std::memory_order_seq_cst)); -} - -} - -template -void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCallback && is_cancelled, TProgressCallback && progress) -{ - from.readPrefix(); - to.writePrefix(); - - while (Block block = from.read()) - { - if (is_cancelled()) - break; - - to.write(block); - progress(block); - } - - if (is_cancelled()) - return; - - /// For outputting additional information in some formats. - if (from.getProfileInfo().hasAppliedLimit()) - to.setRowsBeforeLimit(from.getProfileInfo().getRowsBeforeLimit()); - - to.setTotals(from.getTotals()); - to.setExtremes(from.getExtremes()); - - if (is_cancelled()) - return; - - from.readSuffix(); - to.writeSuffix(); -} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & progress, - std::atomic * is_cancelled) -{ - auto is_cancelled_pred = [is_cancelled] () - { - return isAtomicSet(is_cancelled); - }; - - copyDataImpl(from, to, is_cancelled_pred, progress); -} - -inline void doNothing(const Block &) {} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled) -{ - auto is_cancelled_pred = [is_cancelled] () - { - return isAtomicSet(is_cancelled); - }; - - copyDataImpl(from, to, is_cancelled_pred, doNothing); -} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled) -{ - copyDataImpl(from, to, is_cancelled, doNothing); -} - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled, - const std::function & progress) -{ - copyDataImpl(from, to, is_cancelled, progress); -} - -} diff --git a/src/DataStreams/copyData.h b/src/DataStreams/copyData.h deleted file mode 100644 index 3dc90aed37d..00000000000 --- a/src/DataStreams/copyData.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include - -#include -#include - - -namespace DB -{ - -class Block; - -/** Copies data from the InputStream into the OutputStream - * (for example, from the database to the console, etc.) - */ -void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic * is_cancelled = nullptr); - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & progress, - std::atomic * is_cancelled = nullptr); - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled); - -void copyData(IBlockInputStream & from, IBlockOutputStream & to, const std::function & is_cancelled, - const std::function & progress); - -} diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index f4a5b6b5e4e..9ec8a9523c6 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -14,7 +14,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index cf003dceb8e..cd87cf831a2 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -1,7 +1,6 @@ #include "DictionarySourceHelpers.h" #include #include -#include #include #include #include "DictionaryStructure.h" diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 6cb4d52744a..844a6357e29 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,5 +1,4 @@ #include "HTTPDictionarySource.h" -#include #include #include #include diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ed9f9d52b94..34574ca13f8 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d816ef1d016..f20cec56943 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 85ce83868c6..975075eba96 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -19,7 +19,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1e19c18de43..22ae459a662 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 566ee60a3e6..9d770aee159 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -35,7 +35,6 @@ #include #include -#include #include diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index c785b085a57..b6bb3c5fad5 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 6eb188bce9f..6ffeef5cc7d 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 8a31917caef..6a1a8652b23 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 0d6fe34c0c2..5f44603a420 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 21e15bc74bb..cf24d14b737 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -52,13 +51,6 @@ public: const SelectQueryOptions &, const Names & required_result_column_names_ = Names{}); - /// Read data not from the table specified in the query, but from the prepared source `input`. - InterpreterSelectQuery( - const ASTPtr & query_ptr_, - ContextPtr context_, - const BlockInputStreamPtr & input_, - const SelectQueryOptions & = {}); - /// Read data not from the table specified in the query, but from the prepared pipe `input`. InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index adf1aae3ff3..c191a73bc71 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index bc0aeda56bd..78c4eca5ca6 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -15,7 +15,6 @@ limitations under the License. */ #include #include #include -#include #include diff --git a/src/Interpreters/InterpreterWatchQuery.h b/src/Interpreters/InterpreterWatchQuery.h index 2bc7236582a..e43ed88af2f 100644 --- a/src/Interpreters/InterpreterWatchQuery.h +++ b/src/Interpreters/InterpreterWatchQuery.h @@ -38,8 +38,6 @@ private: /// Table from where to read data, if not subquery. StoragePtr storage; - /// Streams of read data - BlockInputStreams streams; }; } diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index aaa7441b8a4..30115710e22 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index f8402cf0287..fb9f2e25c07 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.h b/src/Interpreters/SortedBlocksWriter.h index c65511e943e..94bebce88f7 100644 --- a/src/Interpreters/SortedBlocksWriter.h +++ b/src/Interpreters/SortedBlocksWriter.h @@ -8,7 +8,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 22cd958f4f8..02dcd95ab41 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index f430c2364b2..0ad40dd3332 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -4,7 +4,6 @@ #include #include #include -#include namespace zkutil diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3f5b386d16f..0a1130c721b 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -10,8 +10,6 @@ #include #include -#include -#include #include #include diff --git a/src/Processors/Formats/OutputStreamToOutputFormat.cpp b/src/Processors/Formats/OutputStreamToOutputFormat.cpp deleted file mode 100644 index 5d4e7832327..00000000000 --- a/src/Processors/Formats/OutputStreamToOutputFormat.cpp +++ /dev/null @@ -1,43 +0,0 @@ -#include -#include - -namespace DB -{ - -Block OutputStreamToOutputFormat::getHeader() const -{ - return output_format->getPort(IOutputFormat::PortKind::Main).getHeader(); -} - -void OutputStreamToOutputFormat::write(const Block & block) -{ - output_format->write(block); -} - -void OutputStreamToOutputFormat::writePrefix() { output_format->doWritePrefix(); } -void OutputStreamToOutputFormat::writeSuffix() { output_format->doWriteSuffix(); } - -void OutputStreamToOutputFormat::flush() { output_format->flush(); } - -void OutputStreamToOutputFormat::setRowsBeforeLimit(size_t rows_before_limit) -{ - output_format->setRowsBeforeLimit(rows_before_limit); -} - -void OutputStreamToOutputFormat::setTotals(const Block & totals) -{ - if (totals) - output_format->setTotals(totals); -} - -void OutputStreamToOutputFormat::setExtremes(const Block & extremes) -{ - if (extremes) - output_format->setExtremes(extremes); -} - -void OutputStreamToOutputFormat::onProgress(const Progress & progress) { output_format->onProgress(progress); } - -std::string OutputStreamToOutputFormat::getContentType() const { return output_format->getContentType(); } - -} diff --git a/src/Processors/Formats/OutputStreamToOutputFormat.h b/src/Processors/Formats/OutputStreamToOutputFormat.h deleted file mode 100644 index a85de12b49d..00000000000 --- a/src/Processors/Formats/OutputStreamToOutputFormat.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once -#include - -namespace DB -{ - - -class IOutputFormat; - -using OutputFormatPtr = std::shared_ptr; - -/// Wrapper. Implements IBlockOutputStream interface using IOutputFormat object. -class OutputStreamToOutputFormat : public IBlockOutputStream -{ -public: - explicit OutputStreamToOutputFormat(OutputFormatPtr output_format_) : output_format(std::move(output_format_)) {} - - Block getHeader() const override; - - void write(const Block & block) override; - - void writePrefix() override; - void writeSuffix() override; - - void flush() override; - - void setRowsBeforeLimit(size_t rows_before_limit) override; - void setTotals(const Block & totals) override; - void setExtremes(const Block & extremes) override; - - void onProgress(const Progress & progress) override; - - std::string getContentType() const override; - -private: - OutputFormatPtr output_format; -}; - -} diff --git a/src/Processors/QueryPipelineBuilder.h b/src/Processors/QueryPipelineBuilder.h index 78ae5dd41be..7e0ddbc0285 100644 --- a/src/Processors/QueryPipelineBuilder.h +++ b/src/Processors/QueryPipelineBuilder.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6776caae9bf..e6ae620e69b 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index eca12c33f54..a5a67e99afc 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 75d0528ff7b..6956dedbc41 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 3720362775c..8f4f04e56c5 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index d001b12ee66..e89d82cfcc8 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -13,7 +13,6 @@ #include #include #include -#include #include "IServer.h" diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6ce17552ba1..4ed3a43d2ed 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 690c9cbd4d0..39688060b0a 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 4641a1631f2..be3dd9ae6c9 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -21,7 +21,6 @@ limitations under the License. */ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 5e12d5da678..d0d3f283478 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f4adee8c259..5dbca837f31 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index d952950e461..e64ba9edec0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 75308f872dc..e3ca902b1bd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -2,9 +2,9 @@ #include #include #include -#include #include #include +#include #include diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 9e138e9882a..947c0bbe932 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -3,7 +3,6 @@ #include "StorageMaterializedPostgreSQL.h" #include #include -#include #include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 873a4b4860c..3796bd8ba57 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 2b4f5e4a276..cf9b557de25 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 59f250d67b8..3e8955ad864 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index d6e242d1a97..0f47f654428 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -11,7 +11,6 @@ #include #include -#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 9aa5689aa66..4ae55272db6 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -16,7 +16,6 @@ #include #include -#include #include #include @@ -479,8 +478,6 @@ Pipe StorageFile::read( size_t max_block_size, unsigned num_streams) { - BlockInputStreams blocks_input; - if (use_table_fd) /// need to call ctr BlockInputStream paths = {""}; /// when use fd, paths are empty else diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..f0c4509f188 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -16,8 +16,6 @@ #include -#include - #include #include diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 5a9e8fc2461..904d1a7f89c 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 846fd4af5fd..063802faf1a 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -8,7 +8,6 @@ #include #include -#include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index eeb5b107b54..15430f60285 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 4264be9dbc2..3bdf3218b2e 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index b4bb5400930..10a60bf9b21 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..20e64255684 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -60,7 +60,6 @@ #include #include -#include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 709c9dc4a63..d95a9465bd6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -30,7 +30,6 @@ #include -#include #include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e6d41a53bfc..0e7faad194e 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 58c56f1401f..2547af1b0ad 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 2dc2577f245..0cd07afc26c 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 79f1d568057..9397986fadd 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -15,7 +15,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a1254e2aaeb..a0924896437 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -1,6 +1,5 @@ #include "StorageXDBC.h" -#include #include #include #include diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 9aedee66b5f..f2b2102c7ff 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index e7ecfc7c4f0..5b891c43aae 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index afd81638da4..49461fe8f46 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -16,7 +16,6 @@ #include -#include namespace DB diff --git a/src/TableFunctions/TableFunctionExecutable.h b/src/TableFunctions/TableFunctionExecutable.h index 05ef2b3b26b..128ee8e46fc 100644 --- a/src/TableFunctions/TableFunctionExecutable.h +++ b/src/TableFunctions/TableFunctionExecutable.h @@ -1,6 +1,5 @@ #pragma once -#include #include namespace DB From 6a0ee3d23ed5710574d134eb76a6989c112c438f Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Oct 2021 07:31:34 -0300 Subject: [PATCH 635/950] test for rename atomic hanging (#30080) * test for rename atomic hanging * test for rename atomic hanging * Update 02096_rename_atomic_hang.sql * Update 02096_rename_atomic_hang.sql Co-authored-by: tavplubix --- .../02096_rename_atomic_hang.reference | 2 ++ .../0_stateless/02096_rename_atomic_hang.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02096_rename_atomic_hang.reference create mode 100644 tests/queries/0_stateless/02096_rename_atomic_hang.sql diff --git a/tests/queries/0_stateless/02096_rename_atomic_hang.reference b/tests/queries/0_stateless/02096_rename_atomic_hang.reference new file mode 100644 index 00000000000..f0073e8f708 --- /dev/null +++ b/tests/queries/0_stateless/02096_rename_atomic_hang.reference @@ -0,0 +1,2 @@ +2000 +2000 diff --git a/tests/queries/0_stateless/02096_rename_atomic_hang.sql b/tests/queries/0_stateless/02096_rename_atomic_hang.sql new file mode 100644 index 00000000000..96261bfe127 --- /dev/null +++ b/tests/queries/0_stateless/02096_rename_atomic_hang.sql @@ -0,0 +1,20 @@ +-- Tags: no-parallel + +drop database if exists db_hang; +drop database if exists db_hang_temp; +create database db_hang engine=Ordinary; +use db_hang; +create table db_hang.test(A Int64) Engine=MergeTree order by A; +create materialized view db_hang.test_mv(A Int64) Engine=MergeTree order by A as select * from db_hang.test; +insert into db_hang.test select * from numbers(1000); + +create database db_hang_temp engine=Atomic; +rename table db_hang.test to db_hang_temp.test; +rename table db_hang.test_mv to db_hang_temp.test_mv; + +drop database db_hang; +rename database db_hang_temp to db_hang; +insert into db_hang.test select * from numbers(1000); +select count() from db_hang.test; +select count() from db_hang.test_mv; +drop database db_hang; From 400a31db06c7cc497d6fde7dbb8f4a4b43efb9d4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 14 Oct 2021 05:06:21 +0000 Subject: [PATCH 636/950] reduce poll timeout fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 5 ++--- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 8 ++++++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 4f6e429e1fd..9ea2e7e1824 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -16,8 +16,7 @@ namespace ErrorCodes extern const int IO_SETUP_ERROR; } -static constexpr int event_size = sizeof(struct inotify_event); -static constexpr int buffer_size = 1024 * (NAME_MAX + event_size + 1); +static constexpr int buffer_size = 4096; DirectoryWatcherBase::DirectoryWatcherBase( FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) @@ -64,7 +63,7 @@ void DirectoryWatcherBase::watchFunc() pfd.events = POLLIN; while (!stopped) { - if (poll(&pfd, 1, 5000) > 0 && pfd.revents & POLLIN) + if (poll(&pfd, 1, 500) > 0 && pfd.revents & POLLIN) { int n = read(fd, buffer.data(), buffer.size()); int i = 0; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 745d8c46a63..f935decd7f8 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -110,16 +110,20 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized", file_meta.file_name); auto & reader = file_ctx.reader.value(); + StorageFileLog::assertStreamGood(reader); + Record record; while (read_records_size < need_records_size && static_cast(reader.tellg()) < file_meta.last_open_end) { /// Need to get offset before reading record from stream record.offset = reader.tellg(); - record.file_name = file_name; - StorageFileLog::assertStreamGood(reader); + record.file_name = file_name; + + std::getline(reader, record.data); + StorageFileLog::assertStreamGood(reader); new_records.emplace_back(record); ++read_records_size; From 4d020c96e0fe2f1725caa6b40354cd7f8014bc4d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Oct 2021 15:07:50 +0300 Subject: [PATCH 637/950] support nullable arguments in function initializeAggregation --- src/Functions/initializeAggregation.cpp | 1 + .../02097_initializeAggregationNullable.reference | 6 ++++++ .../0_stateless/02097_initializeAggregationNullable.sql | 8 ++++++++ 3 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02097_initializeAggregationNullable.reference create mode 100644 tests/queries/0_stateless/02097_initializeAggregationNullable.sql diff --git a/src/Functions/initializeAggregation.cpp b/src/Functions/initializeAggregation.cpp index e8bd136e704..02db90bfc43 100644 --- a/src/Functions/initializeAggregation.cpp +++ b/src/Functions/initializeAggregation.cpp @@ -40,6 +40,7 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; diff --git a/tests/queries/0_stateless/02097_initializeAggregationNullable.reference b/tests/queries/0_stateless/02097_initializeAggregationNullable.reference new file mode 100644 index 00000000000..6d2e42f2ca6 --- /dev/null +++ b/tests/queries/0_stateless/02097_initializeAggregationNullable.reference @@ -0,0 +1,6 @@ +1 +AggregateFunction(uniqExact, Nullable(String)) +1 +AggregateFunction(uniqExact, Nullable(UInt8)) +1 +1 diff --git a/tests/queries/0_stateless/02097_initializeAggregationNullable.sql b/tests/queries/0_stateless/02097_initializeAggregationNullable.sql new file mode 100644 index 00000000000..aa4e6d47579 --- /dev/null +++ b/tests/queries/0_stateless/02097_initializeAggregationNullable.sql @@ -0,0 +1,8 @@ +SELECT finalizeAggregation(initializeAggregation('uniqExactState', toNullable('foo'))); +SELECT toTypeName(initializeAggregation('uniqExactState', toNullable('foo'))); + +SELECT finalizeAggregation(initializeAggregation('uniqExactState', toNullable(123))); +SELECT toTypeName(initializeAggregation('uniqExactState', toNullable(123))); + +SELECT initializeAggregation('uniqExactState', toNullable('foo')) = arrayReduce('uniqExactState', [toNullable('foo')]); +SELECT initializeAggregation('uniqExactState', toNullable(123)) = arrayReduce('uniqExactState', [toNullable(123)]); From ddf6ca716cdbadc617dcf67c6d3e96e5fe5608df Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Oct 2021 09:47:43 -0300 Subject: [PATCH 638/950] Doc. fix optimize_skip_unused_shards description (#30114) * fix optimize_skip_unused_shards description * Update settings.md --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index aa70eb4f721..2b088bf45bf 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1808,7 +1808,7 @@ Default value: 1000 ## optimize_skip_unused_shards {#optimize-skip-unused-shards} -Enables or disables skipping of unused shards for [SELECT](../../sql-reference/statements/select/index.md) queries that have sharding key condition in `WHERE/PREWHERE` (assuming that the data is distributed by sharding key, otherwise does nothing). +Enables or disables skipping of unused shards for [SELECT](../../sql-reference/statements/select/index.md) queries that have sharding key condition in `WHERE/PREWHERE` (assuming that the data is distributed by sharding key, otherwise a query yields incorrect result). Possible values: @@ -3813,4 +3813,4 @@ Default value: `0`. **See Also** -- [optimize_move_to_prewhere](#optimize_move_to_prewhere) setting \ No newline at end of file +- [optimize_move_to_prewhere](#optimize_move_to_prewhere) setting diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index bccbbf69e39..e6e697e5eb6 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1707,7 +1707,7 @@ ClickHouse генерирует исключение ## optimize_skip_unused_shards {#optimize-skip-unused-shards} -Включает или отключает пропуск неиспользуемых шардов для запросов [SELECT](../../sql-reference/statements/select/index.md) , в которых условие ключа шардирования задано в секции `WHERE/PREWHERE`. Предполагается, что данные распределены с помощью ключа шардирования, в противном случае настройка ничего не делает. +Включает или отключает пропуск неиспользуемых шардов для запросов [SELECT](../../sql-reference/statements/select/index.md) , в которых условие ключа шардирования задано в секции `WHERE/PREWHERE`. Предполагается, что данные распределены с помощью ключа шардирования, в противном случае запрос выдаст неверный результат. Возможные значения: @@ -3602,4 +3602,4 @@ SELECT * FROM positional_arguments ORDER BY 2,3; **См. также** -- настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) \ No newline at end of file +- настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) From fbaa9e7be450f3363f543b41b9a5c8a06d770be3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 16:15:39 +0300 Subject: [PATCH 639/950] Support SQL user defined functions for clickhouse-local --- programs/local/LocalServer.cpp | 6 ++++++ .../InterpreterCreateFunctionQuery.cpp | 2 +- src/Interpreters/InterpreterCreateFunctionQuery.h | 6 +++--- src/Interpreters/UserDefinedSQLObjectsLoader.cpp | 14 ++++++++++++++ src/Interpreters/UserDefinedSQLObjectsLoader.h | 4 ++++ 5 files changed, 28 insertions(+), 4 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 8066650006a..92bc51a48da 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -565,6 +565,8 @@ void LocalServer::processConfig() global_context->setCurrentDatabase(default_database); applyCmdOptions(global_context); + bool enable_objects_loader = false; + if (config().has("path")) { String path = global_context->getPath(); @@ -576,6 +578,7 @@ void LocalServer::processConfig() LOG_DEBUG(log, "Loading user defined objects from {}", path); Poco::File(path + "user_defined/").createDirectories(); UserDefinedSQLObjectsLoader::instance().loadObjects(global_context); + enable_objects_loader = true; LOG_DEBUG(log, "Loaded user defined objects."); LOG_DEBUG(log, "Loading metadata from {}", path); @@ -599,6 +602,9 @@ void LocalServer::processConfig() attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); } + /// Persist SQL user defined objects only if user_defined folder was created + UserDefinedSQLObjectsLoader::instance().enable(enable_objects_loader); + server_display_name = config().getString("display_name", getFQDNOrHostName()); prompt_by_server_display_name = config().getRawString("prompt_by_server_display_name.default", "{display_name} :) "); std::map prompt_substitutions{{"display_name", server_display_name}}; diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index b7ed3a2fe75..ccb5f4040ec 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -36,7 +36,7 @@ BlockIO InterpreterCreateFunctionQuery::execute() UserDefinedSQLFunctionFactory::instance().registerFunction(function_name, query_ptr); - if (!is_internal) + if (!persist_function) { try { diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h index b10760c5e9d..fdc03b379db 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.h +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -11,10 +11,10 @@ class Context; class InterpreterCreateFunctionQuery : public IInterpreter, WithContext { public: - InterpreterCreateFunctionQuery(const ASTPtr & query_ptr_, ContextPtr context_, bool is_internal_) + InterpreterCreateFunctionQuery(const ASTPtr & query_ptr_, ContextPtr context_, bool persist_function_) : WithContext(context_) , query_ptr(query_ptr_) - , is_internal(is_internal_) {} + , persist_function(persist_function_) {} BlockIO execute() override; @@ -26,7 +26,7 @@ private: static void validateFunctionRecursiveness(ASTPtr node, const String & function_to_create); ASTPtr query_ptr; - bool is_internal; + bool persist_function; }; } diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp index 455c73bf685..e4eb97f3002 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp @@ -83,6 +83,9 @@ void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, User void UserDefinedSQLObjectsLoader::loadObjects(ContextPtr context) { + if (unlikely(!enable_persistence)) + return; + LOG_DEBUG(log, "loading user defined objects"); String dir_path = context->getPath() + "user_defined/"; @@ -110,6 +113,9 @@ void UserDefinedSQLObjectsLoader::loadObjects(ContextPtr context) void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast) { + if (unlikely(!enable_persistence)) + return; + String dir_path = context->getPath() + "user_defined/"; String file_path; @@ -143,6 +149,9 @@ void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQL void UserDefinedSQLObjectsLoader::removeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name) { + if (unlikely(!enable_persistence)) + return; + String dir_path = context->getPath() + "user_defined/"; LOG_DEBUG(log, "Removing file for user defined object {} from {}", backQuote(object_name), dir_path); @@ -162,4 +171,9 @@ void UserDefinedSQLObjectsLoader::removeObject(ContextPtr context, UserDefinedSQ std::filesystem::remove(file_path); } +void UserDefinedSQLObjectsLoader::enable(bool enable_persistence_) +{ + enable_persistence = enable_persistence_; +} + } diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.h b/src/Interpreters/UserDefinedSQLObjectsLoader.h index 6daf3e99833..17493933f21 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.h +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.h @@ -24,10 +24,14 @@ public: void storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast); void removeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name); + /// For ClickHouse local if path is not set we can disable loader. + void enable(bool enable_persistence); + private: void loadUserDefinedObject(ContextPtr context, UserDefinedSQLObjectType object_type, const std::string_view & object_name, const String & file_path); Poco::Logger * log; + bool enable_persistence = true; }; } From c1f04452de5297553548379295c219966c46370a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 1 Oct 2021 22:35:17 +0300 Subject: [PATCH 640/950] AddDefaultDatabaseVisitor support dictGet --- src/Interpreters/AddDefaultDatabaseVisitor.h | 74 ++++++++++++++----- .../ExternalDictionariesLoader.cpp | 15 ++++ src/Interpreters/ExternalDictionariesLoader.h | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 9 ++- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/SelectQueryDescription.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 1 - 9 files changed, 86 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index 3d93118047f..f4b09d5a761 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -10,10 +10,14 @@ #include #include #include +#include #include #include #include #include +#include +#include +#include namespace DB { @@ -26,11 +30,12 @@ class AddDefaultDatabaseVisitor { public: explicit AddDefaultDatabaseVisitor( - const String & database_name_, bool only_replace_current_database_function_ = false, WriteBuffer * ostr_ = nullptr) - : database_name(database_name_) + ContextPtr context_, + const String & database_name_, + bool only_replace_current_database_function_ = false) + : context(context_) + , database_name(database_name_) , only_replace_current_database_function(only_replace_current_database_function_) - , visit_depth(0) - , ostr(ostr_) {} void visitDDL(ASTPtr & ast) const @@ -64,11 +69,19 @@ public: visit(select, unused); } + void visit(ASTColumns & columns) const + { + for (auto & child : columns.children) + visit(child); + } + private: + + ContextPtr context; + const String database_name; + bool only_replace_current_database_function = false; - mutable size_t visit_depth; - WriteBuffer * ostr; void visit(ASTSelectWithUnionQuery & select, ASTPtr &) const { @@ -117,15 +130,8 @@ private: void visit(ASTFunction & function, ASTPtr &) const { - bool is_operator_in = false; - for (const auto * name : {"in", "notIn", "globalIn", "globalNotIn"}) - { - if (function.name == name) - { - is_operator_in = true; - break; - } - } + bool is_operator_in = functionIsInOrGlobalInOperator(function.name); + bool is_dict_get = functionIsDictGet(function.name); for (auto & child : function.children) { @@ -133,7 +139,38 @@ private: { for (size_t i = 0; i < child->children.size(); ++i) { - if (is_operator_in && i == 1) + if (is_dict_get && i == 0) + { + if (auto * identifier = child->children[i]->as()) + { + if (identifier->compound()) + continue; + + auto storage_id = context->getExternalDictionariesLoader().getStorageID(identifier->name(), context); + + if (!storage_id.database_name.empty()) + { + std::vector name_parts = {storage_id.database_name, storage_id.table_name}; + child->children[i] = std::make_shared(std::move(name_parts)); + } + else + { + std::vector name_parts = {storage_id.table_name}; + child->children[i] = std::make_shared(std::move(name_parts)); + } + } + else if (auto * literal = child->children[i]->as()) + { + auto & literal_value = literal->value; + + if (literal_value.getType() != Field::Types::String) + continue; + + auto dictionary_name = literal_value.get(); + literal_value = context->getExternalDictionariesLoader().getStorageID(dictionary_name, context).getFullTableName(); + } + } + else if (is_operator_in && i == 1) { /// XXX: for some unknown reason this place assumes that argument can't be an alias, /// like in the similar code in `MarkTableIdentifierVisitor`. @@ -151,11 +188,15 @@ private: visit(child->children[i]); } else + { visit(child->children[i]); + } } } else + { visit(child); + } } } @@ -170,7 +211,6 @@ private: { if (T * t = typeid_cast(ast.get())) { - DumpASTNode dump(*ast, ostr, visit_depth, "addDefaultDatabaseName"); visit(*t, ast); return true; } diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index bf2ce9e66ee..c49b037ef96 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -87,6 +87,21 @@ DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); } +StorageID ExternalDictionariesLoader::getStorageID(const std::string & dictionary_name, ContextPtr context) const +{ + if (has(dictionary_name)) + return StorageID("", dictionary_name); + + auto qualified_name = QualifiedTableName::tryParseFromString(dictionary_name); + if (!qualified_name) + return StorageID("", dictionary_name); + + if (qualified_name->database.empty()) + return StorageID(context->getCurrentDatabase(), dictionary_name); + + return StorageID("", dictionary_name); +} + std::string ExternalDictionariesLoader::resolveDictionaryName(const std::string & dictionary_name, const std::string & current_database_name) const { if (has(dictionary_name)) diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index f748d75d908..fea2c363f9a 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -27,6 +27,8 @@ public: void reloadDictionary(const std::string & dictionary_name, ContextPtr context) const; + StorageID getStorageID(const std::string & dictionary_name, ContextPtr context) const; + DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const; static DictionaryStructure getDictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & key_in_config = "dictionary"); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 1c613758ecc..90d5da35df8 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -80,7 +80,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Add default database to table identifiers that we can encounter in e.g. default expressions, mutation expression, etc. - AddDefaultDatabaseVisitor visitor(table_id.getDatabaseName()); + AddDefaultDatabaseVisitor visitor(getContext(), table_id.getDatabaseName()); ASTPtr command_list_ptr = alter.command_list->ptr(); visitor.visit(command_list_ptr); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 530b10703c5..e1b2f9097d6 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -920,10 +920,17 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { // Expand CTE before filling default database ApplyWithSubqueryVisitor().visit(*create.select); - AddDefaultDatabaseVisitor visitor(current_database); + AddDefaultDatabaseVisitor visitor(getContext(), current_database); visitor.visit(*create.select); } + + if (create.columns_list) + { + AddDefaultDatabaseVisitor visitor(getContext(), current_database); + visitor.visit(*create.columns_list); + } + /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create); diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 576c1f3ffdd..45e1c580f01 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -152,7 +152,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context, } } - AddDefaultDatabaseVisitor visitor(current_database, !use_local_default_database); + AddDefaultDatabaseVisitor visitor(context, current_database, !use_local_default_database); visitor.visitDDL(query_ptr); /// Check access rights, assume that all servers have the same users config diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 4641a1631f2..d387c5f5e53 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -70,7 +70,7 @@ static StorageID extractDependentTable(ASTPtr & query, ContextPtr context, const if (db_and_table->database.empty()) { db_and_table->database = select_database_name; - AddDefaultDatabaseVisitor visitor(select_database_name); + AddDefaultDatabaseVisitor visitor(context, select_database_name); visitor.visit(select_query); } else diff --git a/src/Storages/SelectQueryDescription.cpp b/src/Storages/SelectQueryDescription.cpp index 05747a9a260..018a9f0ea98 100644 --- a/src/Storages/SelectQueryDescription.cpp +++ b/src/Storages/SelectQueryDescription.cpp @@ -48,7 +48,7 @@ StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, ContextPt { if (add_default_db) { - AddDefaultDatabaseVisitor visitor(context->getCurrentDatabase(), false, nullptr); + AddDefaultDatabaseVisitor visitor(context, context->getCurrentDatabase()); visitor.visit(query); } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index d2493ff7c43..5436d306122 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include From a1a4df2501816529dcb9e6588acfc72b74b902bc Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 16:34:05 +0300 Subject: [PATCH 641/950] Fix handling exception 'unrecognised option' in clickhouse-local and client --- programs/client/Client.cpp | 10 +--------- programs/client/Client.h | 2 +- programs/local/LocalServer.cpp | 15 +++++---------- programs/local/LocalServer.h | 2 +- src/Client/ClientBase.cpp | 17 ++++++++++++++++- src/Client/ClientBase.h | 3 ++- ...unknown_option_in_clickhouse_local.reference | 3 ++- .../02096_unknown_option_in_clickhouse_local.sh | 2 +- 8 files changed, 29 insertions(+), 25 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index da910430985..d53a57b6eba 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -996,7 +996,7 @@ void Client::printHelpMessage(const OptionsDescription & options_description) } -void Client::addAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +void Client::addOptions(OptionsDescription & options_description) { /// Main commandline options related to client functionality and all parameters from Settings. options_description.main_description->add_options() @@ -1053,14 +1053,6 @@ void Client::addAndCheckOptions(OptionsDescription & options_description, po::va ( "types", po::value(), "types" ); - - cmd_settings.addProgramOptions(options_description.main_description.value()); - /// Parse main commandline options. - po::parsed_options parsed = po::command_line_parser(arguments).options(options_description.main_description.value()).run(); - auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); - if (unrecognized_options.size() > 1) - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[1]); - po::store(parsed, options); } diff --git a/programs/client/Client.h b/programs/client/Client.h index 43f6deae0b5..2def74ef3fc 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -24,7 +24,7 @@ protected: String getName() const override { return "client"; } void printHelpMessage(const OptionsDescription & options_description) override; - void addAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) override; + void addOptions(OptionsDescription & options_description) override; void processOptions(const OptionsDescription & options_description, const CommandLineOptions & options, const std::vector & external_tables_arguments) override; void processConfig() override; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2035406d73a..2180729438d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include @@ -35,7 +34,6 @@ #include #include #include -#include #include #include #include @@ -636,7 +634,7 @@ void LocalServer::printHelpMessage(const OptionsDescription & options_descriptio } -void LocalServer::addAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +void LocalServer::addOptions(OptionsDescription & options_description) { options_description.main_description->add_options() ("database,d", po::value(), "database") @@ -655,10 +653,6 @@ void LocalServer::addAndCheckOptions(OptionsDescription & options_description, p ("no-system-tables", "do not attach system tables (better startup time)") ; - - cmd_settings.addProgramOptions(options_description.main_description.value()); - po::parsed_options parsed = po::command_line_parser(arguments).options(options_description.main_description.value()).run(); - po::store(parsed, options); } @@ -713,10 +707,11 @@ int mainEntryClickHouseLocal(int argc, char ** argv) app.init(argc, argv); return app.run(); } - catch (const boost::program_options::error & e) + catch (const DB::Exception & e) { - std::cerr << "Bad arguments: " << e.what() << std::endl; - return DB::ErrorCodes::BAD_ARGUMENTS; + std::cerr << DB::getExceptionMessage(e, false) << std::endl; + auto code = DB::getCurrentExceptionCode(); + return code ? code : 1; } catch (...) { diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index e14e18adced..ce0df06c86a 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -40,7 +40,7 @@ protected: String getQueryTextPrefix() override; void printHelpMessage(const OptionsDescription & options_description) override; - void addAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) override; + void addOptions(OptionsDescription & options_description) override; void processOptions(const OptionsDescription & options_description, const CommandLineOptions & options, const std::vector &) override; void processConfig() override; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index baf082a3541..deb22ca60ef 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -72,6 +72,7 @@ namespace ErrorCodes extern const int UNEXPECTED_PACKET_FROM_SERVER; extern const int INVALID_USAGE_OF_INPUT; extern const int CANNOT_SET_SIGNAL_HANDLER; + extern const int UNRECOGNIZED_ARGUMENTS; } } @@ -1505,6 +1506,19 @@ void ClientBase::readArguments(int argc, char ** argv, Arguments & common_argume } } +void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) +{ + cmd_settings.addProgramOptions(options_description.main_description.value()); + /// Parse main commandline options. + auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()); + parser.allow_unregistered(); + po::parsed_options parsed = parser.run(); + auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); + if (unrecognized_options.size() > 1) + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[1]); + po::store(parsed, options); +} + void ClientBase::init(int argc, char ** argv) { @@ -1562,7 +1576,8 @@ void ClientBase::init(int argc, char ** argv) ("stacktrace", "print stack traces of exceptions") ; - addAndCheckOptions(options_description, options, common_arguments); + addOptions(options_description); + parseAndCheckOptions(options_description, options, common_arguments); po::notify(options); if (options.count("version") || options.count("V")) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 070b676366c..cfc0b45ff60 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -91,7 +91,7 @@ protected: }; virtual void printHelpMessage(const OptionsDescription & options_description) = 0; - virtual void addAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments) = 0; + virtual void addOptions(OptionsDescription & options_description) = 0; virtual void processOptions(const OptionsDescription & options_description, const CommandLineOptions & options, const std::vector & external_tables_arguments) = 0; @@ -132,6 +132,7 @@ private: void resetOutput(); void outputQueryInfo(bool echo_query_); void readArguments(int argc, char ** argv, Arguments & common_arguments, std::vector & external_tables_arguments); + void parseAndCheckOptions(OptionsDescription & options_description, po::variables_map & options, Arguments & arguments); protected: bool is_interactive = false; /// Use either interactive line editing interface or batch mode. diff --git a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference index 96feda5dd3c..2c4cf540812 100644 --- a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference +++ b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference @@ -1 +1,2 @@ -Bad arguments: unrecognised option '--unknown-option' +Code: 552. DB::Exception: Unrecognized option '--unknown-option'. (UNRECOGNIZED_ARGUMENTS) +Code: 552. DB::Exception: Unrecognized option '--unknown-option'. (UNRECOGNIZED_ARGUMENTS) diff --git a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh index ee0e3f3d149..2fabc761d4c 100755 --- a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh +++ b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh @@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh ${CLICKHOUSE_LOCAL} --unknown-option 2>&1 echo - +${CLICKHOUSE_CLIENT} --unknown-option 2>&1 echo From 2da43012b6203bd9957d099dfbff575420efa4af Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 30 Sep 2021 22:46:12 +0300 Subject: [PATCH 642/950] Add log levels updates --- base/loggers/Loggers.cpp | 91 +++++++++++++++++++++++++++-- base/loggers/Loggers.h | 2 + base/loggers/OwnFormattingChannel.h | 5 +- base/loggers/OwnSplitChannel.cpp | 17 +++++- base/loggers/OwnSplitChannel.h | 6 +- programs/server/Server.cpp | 2 +- 6 files changed, 111 insertions(+), 12 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 80e62d0a6d6..0f41296819e 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -84,7 +84,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, log_file); log->setLevel(log_level); - split->addChannel(log); + split->addChannel(log, "log"); } const auto errorlog_path = config.getString("logger.errorlog", ""); @@ -116,7 +116,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr errorlog = new DB::OwnFormattingChannel(pf, error_log_file); errorlog->setLevel(errorlog_level); errorlog->open(); - split->addChannel(errorlog); + split->addChannel(errorlog, "errorlog"); } if (config.getBool("logger.use_syslog", false)) @@ -155,7 +155,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, syslog_channel); log->setLevel(syslog_level); - split->addChannel(log); + split->addChannel(log, "syslog"); } bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO); @@ -177,7 +177,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel); logger.warning("Logging " + console_log_level_string + " to console"); log->setLevel(console_log_level); - split->addChannel(log); + split->addChannel(log, "console"); } split->open(); @@ -224,6 +224,89 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log } } +void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger) +{ + int max_log_level = 0; + + const auto log_level_string = config.getString("logger.level", "trace"); + int log_level = Poco::Logger::parseLevel(log_level_string); + if (log_level > max_log_level) + max_log_level = log_level; + + const auto log_path = config.getString("logger.log", ""); + if (!log_path.empty()) + split->setLevel("log", log_level); + else + split->setLevel("log", 0); + + // Set level to console + bool is_daemon = config.getBool("application.runAsDaemon", false); + bool should_log_to_console = isatty(STDIN_FILENO) || isatty(STDERR_FILENO); + if (config.getBool("logger.console", false) + || (!config.hasProperty("logger.console") && !is_daemon && should_log_to_console)) + split->setLevel("console", log_level); + else + split->setLevel("console", 0); + + // Set level to errorlog + int errorlog_level = 0; + const auto errorlog_path = config.getString("logger.errorlog", ""); + if (!errorlog_path.empty()) + { + errorlog_level = Poco::Logger::parseLevel(config.getString("logger.errorlog_level", "notice")); + if (errorlog_level > max_log_level) + max_log_level = errorlog_level; + } + split->setLevel("errorlog", errorlog_level); + + // Set level to syslog + int syslog_level = 0; + if (config.getBool("logger.use_syslog", false)) + { + syslog_level = Poco::Logger::parseLevel(config.getString("logger.syslog_level", log_level_string)); + if (syslog_level > max_log_level) + max_log_level = syslog_level; + } + split->setLevel("syslog", syslog_level); + + // Global logging level (it can be overridden for specific loggers). + logger.setLevel(max_log_level); + + // Set level to all already created loggers + std::vector names; + + logger.root().names(names); + for (const auto & name : names) + logger.root().get(name).setLevel(max_log_level); + + logger.root().setLevel(max_log_level); + + // Explicitly specified log levels for specific loggers. + { + Poco::Util::AbstractConfiguration::Keys loggers_level; + config.keys("logger.levels", loggers_level); + + if (!loggers_level.empty()) + { + for (const auto & key : loggers_level) + { + if (key == "logger" || key.starts_with("logger[")) + { + const std::string name(config.getString("logger.levels." + key + ".name")); + const std::string level(config.getString("logger.levels." + key + ".level")); + logger.root().get(name).setLevel(level); + } + else + { + // Legacy syntax + const std::string level(config.getString("logger.levels." + key, "trace")); + logger.root().get(key).setLevel(level); + } + } + } + } +} + void Loggers::closeLogs(Poco::Logger & logger) { if (log_file) diff --git a/base/loggers/Loggers.h b/base/loggers/Loggers.h index 151c1d3566f..e8afd749534 100644 --- a/base/loggers/Loggers.h +++ b/base/loggers/Loggers.h @@ -19,6 +19,8 @@ class Loggers public: void buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger, const std::string & cmd_name = ""); + void updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger); + /// Close log files. On next log write files will be reopened. void closeLogs(Poco::Logger & logger); diff --git a/base/loggers/OwnFormattingChannel.h b/base/loggers/OwnFormattingChannel.h index 2336dacad04..0480d0d5061 100644 --- a/base/loggers/OwnFormattingChannel.h +++ b/base/loggers/OwnFormattingChannel.h @@ -1,4 +1,5 @@ #pragma once +#include #include #include #include @@ -14,7 +15,7 @@ class OwnFormattingChannel : public Poco::Channel, public ExtendedLogChannel public: explicit OwnFormattingChannel( Poco::AutoPtr pFormatter_ = nullptr, Poco::AutoPtr pChannel_ = nullptr) - : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)) + : pFormatter(std::move(pFormatter_)), pChannel(std::move(pChannel_)), priority(Poco::Message::PRIO_TRACE) { } @@ -45,7 +46,7 @@ public: private: Poco::AutoPtr pFormatter; Poco::AutoPtr pChannel; - Poco::Message::Priority priority = Poco::Message::PRIO_TRACE; + std::atomic priority; }; } diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 2349c60856f..2ae1e65729c 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -1,4 +1,5 @@ #include "OwnSplitChannel.h" +#include "OwnFormattingChannel.h" #include #include @@ -75,7 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) ExtendedLogMessage msg_ext = ExtendedLogMessage::getFrom(msg); /// Log data to child channels - for (auto & channel : channels) + for (auto & [name, channel] : channels) { if (channel.second) channel.second->logExtended(msg_ext); // extended child @@ -137,9 +138,9 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) } -void OwnSplitChannel::addChannel(Poco::AutoPtr channel) +void OwnSplitChannel::addChannel(Poco::AutoPtr channel, const std::string & name) { - channels.emplace_back(std::move(channel), dynamic_cast(channel.get())); + channels.emplace(name, ExtendedChannelPtrPair(std::move(channel), dynamic_cast(channel.get()))); } void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_priority) @@ -149,4 +150,14 @@ void OwnSplitChannel::addTextLog(std::shared_ptr log, int max_prior text_log_max_priority.store(max_priority, std::memory_order_relaxed); } +void OwnSplitChannel::setLevel(const std::string & name, int level) +{ + auto it = channels.find(name); + if (it != channels.end()) + { + if (auto * channel = dynamic_cast(it->second.first.get())) + channel->setLevel(level); + } +} + } diff --git a/base/loggers/OwnSplitChannel.h b/base/loggers/OwnSplitChannel.h index 03ff7b57745..fdc580e65f8 100644 --- a/base/loggers/OwnSplitChannel.h +++ b/base/loggers/OwnSplitChannel.h @@ -18,10 +18,12 @@ public: /// Makes an extended message from msg and passes it to the client logs queue and child (if possible) void log(const Poco::Message & msg) override; /// Adds a child channel - void addChannel(Poco::AutoPtr channel); + void addChannel(Poco::AutoPtr channel, const std::string & name); void addTextLog(std::shared_ptr log, int max_priority); + void setLevel(const std::string & name, int level); + private: void logSplit(const Poco::Message & msg); void tryLogSplit(const Poco::Message & msg); @@ -29,7 +31,7 @@ private: using ChannelPtr = Poco::AutoPtr; /// Handler and its pointer casted to extended interface using ExtendedChannelPtrPair = std::pair; - std::vector channels; + std::map channels; std::mutex text_log_mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 01033570926..bfb77f49763 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -844,7 +844,7 @@ if (ThreadFuzzer::instance().isEffective()) // FIXME logging-related things need synchronization -- see the 'Logger * log' saved // in a lot of places. For now, disable updating log configuration without server restart. //setTextLog(global_context->getTextLog()); - //buildLoggers(*config, logger()); + updateLevels(*config, logger()); global_context->setClustersConfig(config); global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); From f29261741c2dcc45866a76b89a9176abb1482f44 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 4 Oct 2021 12:25:42 +0300 Subject: [PATCH 643/950] Add test --- .../test_log_levels_update/__init__.py | 0 .../test_log_levels_update/configs/log.xml | 6 +++ .../test_log_levels_update/test.py | 50 +++++++++++++++++++ 3 files changed, 56 insertions(+) create mode 100644 tests/integration/test_log_levels_update/__init__.py create mode 100644 tests/integration/test_log_levels_update/configs/log.xml create mode 100644 tests/integration/test_log_levels_update/test.py diff --git a/tests/integration/test_log_levels_update/__init__.py b/tests/integration/test_log_levels_update/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_log_levels_update/configs/log.xml b/tests/integration/test_log_levels_update/configs/log.xml new file mode 100644 index 00000000000..668a15f6afd --- /dev/null +++ b/tests/integration/test_log_levels_update/configs/log.xml @@ -0,0 +1,6 @@ + + + trace + /var/log/clickhouse-server/clickhouse-server.log + + \ No newline at end of file diff --git a/tests/integration/test_log_levels_update/test.py b/tests/integration/test_log_levels_update/test.py new file mode 100644 index 00000000000..dca660a2982 --- /dev/null +++ b/tests/integration/test_log_levels_update/test.py @@ -0,0 +1,50 @@ +import pytest +import re + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__, name="log_quries_probability") +node = cluster.add_instance('node', with_zookeeper=False) + +config = ''' + + information + /var/log/clickhouse-server/clickhouse-server.log + +''' + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def get_log(node): + return node.exec_in_container(["bash", "-c", "cat /var/log/clickhouse-server/clickhouse-server.log"]) + +def test_log_levels_update(start_cluster): + # Make sure that there are enough log messages for the test + for i in range(5): + node.query("SELECT 1") + + log = get_log(node) + assert re.search("(|)", log) + + node.replace_config("/etc/clickhouse-server/config.d/log.xml", config) + node.query("SYSTEM RELOAD CONFIG;") + node.exec_in_container(["bash", "-c", "> /var/log/clickhouse-server/clickhouse-server.log"]) + + for i in range(5): + node.query("SELECT 1") + + log = get_log(node) + assert len(log) > 0 + assert not re.search("(|)", log) + + + From ba7b784646bc64619dc62d72c3d27e47e457949f Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 16:36:32 +0300 Subject: [PATCH 644/950] Remove catching boost::program_options error in Client --- programs/client/Client.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index d53a57b6eba..45314a5d460 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1230,11 +1230,6 @@ int mainEntryClickHouseClient(int argc, char ** argv) client.init(argc, argv); return client.run(); } - catch (const boost::program_options::error & e) - { - std::cerr << "Bad arguments: " << e.what() << std::endl; - return 1; - } catch (const DB::Exception & e) { std::cerr << DB::getExceptionMessage(e, false) << std::endl; From 04e07ed86d277b5106443b73d58380755d05dc01 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 13 Oct 2021 20:11:26 +0300 Subject: [PATCH 645/950] FunctionsJSON avoid copying object element during iteration --- src/Functions/FunctionsJSON.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 4fd4e6b2d00..7dd724dbfc9 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -1001,7 +1001,7 @@ struct JSONExtractTree } else { - for (auto [key, value] : object) + for (auto & [key, value] : object) { auto index = name_to_index_map.find(key); if (index != name_to_index_map.end()) @@ -1164,7 +1164,7 @@ public: auto & col_key = assert_cast(col_tuple.getColumn(0)); auto & col_value = col_tuple.getColumn(1); - for (auto [key, value] : object) + for (auto & [key, value] : object) { if (extract_tree->insertResultToColumn(col_value, value)) col_key.insertData(key.data(), key.size()); @@ -1343,7 +1343,7 @@ public: auto & col_key = assert_cast(col_tuple.getColumn(0)); auto & col_value = assert_cast(col_tuple.getColumn(1)); - for (auto [key, value] : object) + for (auto & [key, value] : object) { col_key.insertData(key.data(), key.size()); JSONExtractRawImpl::insertResultToColumn(col_value, value, {}); From ce22f534c4e64d7c4fe13c3fb1353c76028aa4e7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 28 Sep 2021 15:59:22 +0300 Subject: [PATCH 646/950] Add CapnProto output format, refactor CapnProto input format --- src/Common/ErrorCodes.cpp | 2 + src/Core/Settings.h | 3 +- src/Core/SettingsEnums.cpp | 5 + src/Core/SettingsEnums.h | 2 + src/DataTypes/EnumValues.cpp | 19 + src/DataTypes/EnumValues.h | 4 + src/Formats/CapnProtoUtils.cpp | 406 ++++++++++++++++ src/Formats/CapnProtoUtils.h | 43 ++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSchemaInfo.cpp | 6 + src/Formats/FormatSchemaInfo.h | 2 + src/Formats/FormatSettings.h | 14 + src/Formats/registerFormats.cpp | 2 + .../Formats/Impl/CapnProtoRowInputFormat.cpp | 433 +++++++++--------- .../Formats/Impl/CapnProtoRowInputFormat.h | 46 +- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 251 ++++++++++ .../Formats/Impl/CapnProtoRowOutputFormat.h | 53 +++ .../Formats/Impl/ProtobufRowInputFormat.cpp | 3 +- .../Formats/Impl/ProtobufRowOutputFormat.cpp | 4 +- .../0_stateless/02030_capnp_format.reference | 52 +++ .../queries/0_stateless/02030_capnp_format.sh | 109 +++++ .../format_schemas/02030_capnp_enum.capnp | 13 + .../02030_capnp_fake_nullable.capnp | 23 + .../format_schemas/02030_capnp_lists.capnp | 8 + .../02030_capnp_low_cardinality.capnp | 17 + .../02030_capnp_nested_lists_and_tuples.capnp | 36 ++ .../02030_capnp_nested_table.capnp | 20 + .../02030_capnp_nested_tuples.capnp | 23 + .../format_schemas/02030_capnp_nullable.capnp | 22 + .../02030_capnp_simple_types.capnp | 21 + .../format_schemas/02030_capnp_tuples.capnp | 35 ++ .../02030_capnp_unnamed_union.capnp | 10 + 32 files changed, 1416 insertions(+), 272 deletions(-) create mode 100644 src/Formats/CapnProtoUtils.cpp create mode 100644 src/Formats/CapnProtoUtils.h create mode 100644 src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h create mode 100644 tests/queries/0_stateless/02030_capnp_format.reference create mode 100755 tests/queries/0_stateless/02030_capnp_format.sh create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_enum.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_fake_nullable.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_lists.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_low_cardinality.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_nested_lists_and_tuples.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_nested_table.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_nested_tuples.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_nullable.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_simple_types.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_tuples.capnp create mode 100644 tests/queries/0_stateless/format_schemas/02030_capnp_unnamed_union.capnp diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b6d9b65c28b..1aff1460125 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,6 +589,8 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(621, CANNOT_NORMALIZE_STRING) \ + M(622, CANNOT_PARSE_CAPN_PROTO_SCHEMA) \ + M(623, CAPN_PROTO_BAD_CAST) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a5767955045..f91bf684c85 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -625,7 +625,8 @@ class IColumn; M(Bool, cross_to_inner_join_rewrite, true, "Use inner join instead of comma/cross join if possible", 0) \ \ M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \ - + \ + M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0)\ // End of FORMAT_FACTORY_SETTINGS // Please add settings non-related to formats into the COMMON_SETTINGS above. diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 8e588b62326..f5497588891 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -116,4 +116,9 @@ IMPLEMENT_SETTING_ENUM(ShortCircuitFunctionEvaluation, ErrorCodes::BAD_ARGUMENTS {{"enable", ShortCircuitFunctionEvaluation::ENABLE}, {"force_enable", ShortCircuitFunctionEvaluation::FORCE_ENABLE}, {"disable", ShortCircuitFunctionEvaluation::DISABLE}}) + +IMPLEMENT_SETTING_ENUM(EnumComparingMode, ErrorCodes::BAD_ARGUMENTS, + {{"by_names", FormatSettings::EnumComparingMode::BY_NAMES}, + {"by_values", FormatSettings::EnumComparingMode::BY_VALUES}, + {"by_names_case_insensitive", FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index a699da3062c..f57a064241e 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -168,4 +168,6 @@ enum class ShortCircuitFunctionEvaluation DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) +DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) + } diff --git a/src/DataTypes/EnumValues.cpp b/src/DataTypes/EnumValues.cpp index 6df899ba9a2..ab5ea0ca249 100644 --- a/src/DataTypes/EnumValues.cpp +++ b/src/DataTypes/EnumValues.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -82,6 +83,24 @@ Names EnumValues::getAllRegisteredNames() const return result; } +template +std::unordered_set EnumValues::getSetOfAllNames(bool to_lower) const +{ + std::unordered_set result; + for (const auto & value : values) + result.insert(to_lower ? boost::algorithm::to_lower_copy(value.first) : value.first); + return result; +} + +template +std::unordered_set EnumValues::getSetOfAllValues() const +{ + std::unordered_set result; + for (const auto & value : values) + result.insert(value.second); + return result; +} + template class EnumValues; template class EnumValues; diff --git a/src/DataTypes/EnumValues.h b/src/DataTypes/EnumValues.h index 1e5e4f55ea7..17c292c5551 100644 --- a/src/DataTypes/EnumValues.h +++ b/src/DataTypes/EnumValues.h @@ -80,6 +80,10 @@ public: } Names getAllRegisteredNames() const override; + + std::unordered_set getSetOfAllNames(bool to_lower) const; + + std::unordered_set getSetOfAllValues() const; }; } diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp new file mode 100644 index 00000000000..9931785f43e --- /dev/null +++ b/src/Formats/CapnProtoUtils.cpp @@ -0,0 +1,406 @@ +#include + +#if USE_CAPNP + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_CAPN_PROTO_SCHEMA; + extern const int THERE_IS_NO_COLUMN; + extern const int BAD_TYPE_OF_FIELD; + extern const int CAPN_PROTO_BAD_CAST; + extern const int FILE_DOESNT_EXIST; + extern const int UNKNOWN_EXCEPTION; +} + +capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) +{ + capnp::ParsedSchema schema; + try + { + int fd; + KJ_SYSCALL(fd = open(schema_info.schemaDirectory().data(), O_RDONLY)); + auto schema_dir = kj::newDiskDirectory(kj::OsFileHandle(fd)); + schema = impl.parseFromDirectory(*schema_dir, kj::Path::parse(schema_info.schemaPath()), {}); + } + catch (const kj::Exception & e) + { + /// That's not good to determine the type of error by its description, but + /// this is the only way to do it here, because kj doesn't specify the type of error. + String description = String(e.getDescription().cStr()); + if (description.starts_with("no such directory")) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); + + if (description.starts_with("Parse error")) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); + + throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception while parsing CapnProro schema: {}, schema dir and file: {}, {}", description, schema_info.schemaDirectory(), schema_info.schemaPath()); + } + + auto message_maybe = schema.findNested(schema_info.messageName()); + auto * message_schema = kj::_::readMaybe(message_maybe); + if (!message_schema) + throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "CapnProto schema doesn't contain message with name {}", schema_info.messageName()); + return message_schema->asStruct(); +} + +bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode) +{ + if (mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE) + return boost::algorithm::to_lower_copy(first) == boost::algorithm::to_lower_copy(second); + return first == second; +} + +static const std::map capnp_simple_type_names = +{ + {capnp::schema::Type::Which::BOOL, "Bool"}, + {capnp::schema::Type::Which::VOID, "Void"}, + {capnp::schema::Type::Which::INT8, "Int8"}, + {capnp::schema::Type::Which::INT16, "Int16"}, + {capnp::schema::Type::Which::INT32, "Int32"}, + {capnp::schema::Type::Which::INT64, "Int64"}, + {capnp::schema::Type::Which::UINT8, "UInt8"}, + {capnp::schema::Type::Which::UINT16, "UInt16"}, + {capnp::schema::Type::Which::UINT32, "UInt32"}, + {capnp::schema::Type::Which::UINT64, "UInt64"}, + {capnp::schema::Type::Which::FLOAT32, "Float32"}, + {capnp::schema::Type::Which::FLOAT64, "Float64"}, + {capnp::schema::Type::Which::TEXT, "Text"}, + {capnp::schema::Type::Which::DATA, "Data"}, + {capnp::schema::Type::Which::ENUM, "Enum"}, + {capnp::schema::Type::Which::INTERFACE, "Interface"}, + {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, +}; + +static bool checkIfStructContainsUnnamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() != struct_schema.getNonUnionFields().size(); +} + +static bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) +{ + return struct_schema.getFields().size() == struct_schema.getUnionFields().size(); +} + +/// Get full name of type for better exception messages. +static String getCapnProtoFullTypeName(const capnp::Type & type) +{ + if (type.isStruct()) + { + auto struct_schema = type.asStruct(); + + auto non_union_fields = struct_schema.getNonUnionFields(); + std::vector non_union_field_names; + for (auto nested_field : non_union_fields) + non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + auto union_fields = struct_schema.getUnionFields(); + std::vector union_field_names; + for (auto nested_field : union_fields) + union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + + String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; + /// Check if the struct is a named union. + if (non_union_field_names.empty()) + return union_name; + + String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); + /// Check if the struct contains unnamed union. + if (!union_field_names.empty()) + type_name += "," + union_name; + type_name += ")"; + return type_name; + } + + if (type.isList()) + return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; + + if (!capnp_simple_type_names.contains(type.which())) + throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); + + return capnp_simple_type_names.at(type.which()); +} + +template +static bool checkEnums(const capnp::Type & capnp_type, const DataTypePtr column_type, FormatSettings::EnumComparingMode mode, UInt64 max_value, String & error_message) +{ + if (!capnp_type.isEnum()) + return false; + + auto enum_schema = capnp_type.asEnum(); + bool to_lower = mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE; + const auto * enum_type = assert_cast *>(column_type.get()); + const auto & enum_values = dynamic_cast &>(*enum_type); + + auto names = enum_values.getSetOfAllNames(to_lower); + auto values = enum_values.getSetOfAllValues(); + + std::unordered_set capn_enum_names; + std::unordered_set capn_enum_values; + + auto enumerants = enum_schema.getEnumerants(); + for (auto enumerant : enumerants) + { + String name = enumerant.getProto().getName(); + capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); + auto value = enumerant.getOrdinal(); + if (mode == FormatSettings::EnumComparingMode::BY_VALUES && value > max_value) + { + error_message += "Enum from CapnProto schema contains value that is out of range for Clickhouse Enum"; + return false; + } + capn_enum_values.insert(Type(value)); + } + + if (mode == FormatSettings::EnumComparingMode::BY_NAMES || mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE) + { + auto result = names == capn_enum_names; + if (!result) + error_message += "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"; + return result; + } + + auto result = values == capn_enum_values; + if (!result) + error_message += "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"; + return result; +} + +static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message); + +static bool checkNullableType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) +{ + if (!capnp_type.isStruct()) + return false; + + /// Check that struct is a named union of type VOID and one arbitrary type. + auto struct_schema = capnp_type.asStruct(); + if (!checkIfStructIsNamedUnion(struct_schema)) + return false; + + auto union_fields = struct_schema.getUnionFields(); + if (union_fields.size() != 2) + return false; + + auto first = union_fields[0]; + auto second = union_fields[1]; + + auto nested_type = assert_cast(data_type.get())->getNestedType(); + if (first.getType().isVoid()) + return checkCapnProtoType(second.getType(), nested_type, mode, error_message); + if (second.getType().isVoid()) + return checkCapnProtoType(first.getType(), nested_type, mode, error_message); + return false; +} + +static bool checkTupleType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) +{ + if (!capnp_type.isStruct()) + return false; + auto struct_schema = capnp_type.asStruct(); + + if (checkIfStructIsNamedUnion(struct_schema)) + return false; + + if (checkIfStructContainsUnnamedUnion(struct_schema)) + { + error_message += "CapnProto struct contains unnamed union"; + return false; + } + + const auto * tuple_data_type = assert_cast(data_type.get()); + auto nested_types = tuple_data_type->getElements(); + if (nested_types.size() != struct_schema.getFields().size()) + { + error_message += "Tuple and Struct types have different sizes"; + return false; + } + + if (!tuple_data_type->haveExplicitNames()) + { + error_message += "Only named Tuple can be converted to CapnProto Struct"; + return false; + } + for (const auto & name : tuple_data_type->getElementNames()) + { + KJ_IF_MAYBE(field, struct_schema.findFieldByName(name)) + { + if (!checkCapnProtoType(field->getType(), nested_types[tuple_data_type->getPositionByName(name)], mode, error_message)) + return false; + } + else + { + error_message += "CapnProto struct doesn't contain a field with name " + name; + return false; + } + } + + return true; +} + +static bool checkArrayType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) +{ + if (!capnp_type.isList()) + return false; + auto list_schema = capnp_type.asList(); + auto nested_type = assert_cast(data_type.get())->getNestedType(); + return checkCapnProtoType(list_schema.getElementType(), nested_type, mode, error_message); +} + +static bool checkCapnProtoType(const capnp::Type & capnp_type, const DataTypePtr & data_type, FormatSettings::EnumComparingMode mode, String & error_message) +{ + switch (data_type->getTypeId()) + { + case TypeIndex::UInt8: + return capnp_type.isBool() || capnp_type.isUInt8(); + case TypeIndex::Date: [[fallthrough]]; + case TypeIndex::UInt16: + return capnp_type.isUInt16(); + case TypeIndex::DateTime: [[fallthrough]]; + case TypeIndex::UInt32: + return capnp_type.isUInt32(); + case TypeIndex::UInt64: + return capnp_type.isUInt64(); + case TypeIndex::Int8: + return capnp_type.isInt8(); + case TypeIndex::Int16: + return capnp_type.isInt16(); + case TypeIndex::Date32: [[fallthrough]]; + case TypeIndex::Int32: + return capnp_type.isInt32(); + case TypeIndex::DateTime64: [[fallthrough]]; + case TypeIndex::Int64: + return capnp_type.isInt64(); + case TypeIndex::Float32: + return capnp_type.isFloat32(); + case TypeIndex::Float64: + return capnp_type.isFloat64(); + case TypeIndex::Enum8: + return checkEnums(capnp_type, data_type, mode, INT8_MAX, error_message); + case TypeIndex::Enum16: + return checkEnums(capnp_type, data_type, mode, INT16_MAX, error_message); + case TypeIndex::Tuple: + return checkTupleType(capnp_type, data_type, mode, error_message); + case TypeIndex::Nullable: + { + auto result = checkNullableType(capnp_type, data_type, mode, error_message); + if (!result) + error_message += "Nullable can be represented only as a named union of type Void and nested type"; + return result; + } + case TypeIndex::Array: + return checkArrayType(capnp_type, data_type, mode, error_message); + case TypeIndex::LowCardinality: + return checkCapnProtoType(capnp_type, assert_cast(data_type.get())->getDictionaryType(), mode, error_message); + case TypeIndex::FixedString: [[fallthrough]]; + case TypeIndex::String: + return capnp_type.isText() || capnp_type.isData(); + default: + return false; + } +} + +static std::pair splitFieldName(const String & name) +{ + const auto * begin = name.data(); + const auto * end = name.data() + name.size(); + const auto * it = find_first_symbols<'_', '.'>(begin, end); + String first = String(begin, it); + String second = it == end ? "" : String(it + 1, end); + return {first, second}; +} + +capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name) +{ + auto [field_name, nested_name] = splitFieldName(name); + KJ_IF_MAYBE(field, struct_reader.getSchema().findFieldByName(field_name)) + { + auto field_reader = struct_reader.get(*field); + if (nested_name.empty()) + return field_reader; + + if (field_reader.getType() != capnp::DynamicValue::STRUCT) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); + + return getReaderByColumnName(field_reader.as(), nested_name); + } + + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); +} + +std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name) +{ + auto [field_name, nested_name] = splitFieldName(name); + KJ_IF_MAYBE(field, struct_builder.getSchema().findFieldByName(field_name)) + { + if (nested_name.empty()) + return {struct_builder, *field}; + + auto field_builder = struct_builder.get(*field); + if (field_builder.getType() != capnp::DynamicValue::STRUCT) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); + + return getStructBuilderAndFieldByColumnName(field_builder.as(), nested_name); + } + + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto struct doesn't contain field with name {}", field_name); +} + +static capnp::StructSchema::Field getFieldByName(const capnp::StructSchema & schema, const String & name) +{ + auto [field_name, nested_name] = splitFieldName(name); + KJ_IF_MAYBE(field, schema.findFieldByName(field_name)) + { + if (nested_name.empty()) + return *field; + + if (!field->getType().isStruct()) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Field {} is not a struct", field_name); + + return getFieldByName(field->getType().asStruct(), nested_name); + } + + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, "Capnproto schema doesn't contain field with name {}", field_name); +} + +void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode) +{ + /// Firstly check that struct doesn't contain unnamed union, because we don't support it. + if (checkIfStructContainsUnnamedUnion(schema)) + throw Exception(ErrorCodes::CAPN_PROTO_BAD_CAST, "Schema contains unnamed union that is not supported"); + auto names_and_types = header.getNamesAndTypesList(); + String additional_error_message; + for (auto & [name, type] : names_and_types) + { + auto field = getFieldByName(schema, name); + if (!checkCapnProtoType(field.getType(), type, mode, additional_error_message)) + { + auto e = Exception( + ErrorCodes::CAPN_PROTO_BAD_CAST, + "Cannot convert ClickHouse type {} to CapnProto type {}", + type->getName(), + getCapnProtoFullTypeName(field.getType())); + if (!additional_error_message.empty()) + e.addMessage(additional_error_message); + throw std::move(e); + } + } +} + +} + +#endif diff --git a/src/Formats/CapnProtoUtils.h b/src/Formats/CapnProtoUtils.h new file mode 100644 index 00000000000..93ca0a5e616 --- /dev/null +++ b/src/Formats/CapnProtoUtils.h @@ -0,0 +1,43 @@ +#pragma once + +#include "config_formats.h" +#if USE_CAPNP + +#include +#include +#include +#include +#include + +namespace DB +{ +// Wrapper for classes that could throw in destructor +// https://github.com/capnproto/capnproto/issues/553 +template +struct DestructorCatcher +{ + T impl; + template + DestructorCatcher(Arg && ... args) : impl(kj::fwd(args)...) {} + ~DestructorCatcher() noexcept try { } catch (...) { return; } +}; + +class CapnProtoSchemaParser : public DestructorCatcher +{ +public: + CapnProtoSchemaParser() {} + + capnp::StructSchema getMessageSchema(const FormatSchemaInfo & schema_info); +}; + +bool compareEnumNames(const String & first, const String & second, FormatSettings::EnumComparingMode mode); + +std::pair getStructBuilderAndFieldByColumnName(capnp::DynamicStruct::Builder struct_builder, const String & name); + +capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Reader & struct_reader, const String & name); + +void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode); + +} + +#endif diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d3ff5cbf8a7..63cb26ab87c 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -112,6 +112,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; format_settings.orc.import_nested = settings.input_format_orc_import_nested; + format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSchemaInfo.cpp b/src/Formats/FormatSchemaInfo.cpp index 2605c0bdf04..24c8dfc14f2 100644 --- a/src/Formats/FormatSchemaInfo.cpp +++ b/src/Formats/FormatSchemaInfo.cpp @@ -99,4 +99,10 @@ FormatSchemaInfo::FormatSchemaInfo(const String & format_schema, const String & } } +FormatSchemaInfo::FormatSchemaInfo(const FormatSettings & settings, const String & format, bool require_message) + : FormatSchemaInfo( + settings.schema.format_schema, format, require_message, settings.schema.is_server, settings.schema.format_schema_path) +{ +} + } diff --git a/src/Formats/FormatSchemaInfo.h b/src/Formats/FormatSchemaInfo.h index cb041e02116..8c430218af0 100644 --- a/src/Formats/FormatSchemaInfo.h +++ b/src/Formats/FormatSchemaInfo.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -11,6 +12,7 @@ class FormatSchemaInfo { public: FormatSchemaInfo(const String & format_schema, const String & format, bool require_message, bool is_server, const std::string & format_schema_path); + FormatSchemaInfo(const FormatSettings & settings, const String & format, bool require_message); /// Returns path to the schema file. const String & schemaPath() const { return schema_path; } diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8c894c77e82..ce5f1effa8c 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -183,6 +183,20 @@ struct FormatSettings { bool import_nested = false; } orc; + + /// For apnProto format we should determine how to + /// compare ClickHouse Enum and Enum from schema. + enum class EnumComparingMode + { + BY_NAMES, // Names in enums should be the same, values can be different. + BY_NAMES_CASE_INSENSITIVE, // Case-insensitive name comparison. + BY_VALUES, // Values should be the same, names can be different. + }; + + struct + { + EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES; + } capn_proto; }; } diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 3e4c0366e8a..f6b4bb7e2e1 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -67,6 +67,7 @@ void registerOutputFormatNull(FormatFactory & factory); void registerOutputFormatMySQLWire(FormatFactory & factory); void registerOutputFormatMarkdown(FormatFactory & factory); void registerOutputFormatPostgreSQLWire(FormatFactory & factory); +void registerOutputFormatCapnProto(FormatFactory & factory); /// Input only formats. @@ -139,6 +140,7 @@ void registerFormats() registerOutputFormatMySQLWire(factory); registerOutputFormatMarkdown(factory); registerOutputFormatPostgreSQLWire(factory); + registerOutputFormatProcessorsCapnProto(factory); registerInputFormatRegexp(factory); registerInputFormatJSONAsString(factory); diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index fd4b2870bea..8492fc9b623 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -1,7 +1,6 @@ #include "CapnProtoRowInputFormat.h" #if USE_CAPNP -#include #include #include #include @@ -9,198 +8,40 @@ #include #include #include -#include -#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include namespace DB { namespace ErrorCodes { - extern const int BAD_TYPE_OF_FIELD; - extern const int THERE_IS_NO_COLUMN; extern const int LOGICAL_ERROR; } -static CapnProtoRowInputFormat::NestedField split(const Block & header, size_t i) -{ - CapnProtoRowInputFormat::NestedField field = {{}, i}; - - // Remove leading dot in field definition, e.g. ".msg" -> "msg" - String name(header.safeGetByPosition(i).name); - if (!name.empty() && name[0] == '.') - name.erase(0, 1); - - splitInto<'.', '_'>(field.tokens, name); - return field; -} - - -static Field convertNodeToField(const capnp::DynamicValue::Reader & value) -{ - switch (value.getType()) - { - case capnp::DynamicValue::UNKNOWN: - throw Exception("Unknown field type", ErrorCodes::BAD_TYPE_OF_FIELD); - case capnp::DynamicValue::VOID: - return Field(); - case capnp::DynamicValue::BOOL: - return value.as() ? 1u : 0u; - case capnp::DynamicValue::INT: - return value.as(); - case capnp::DynamicValue::UINT: - return value.as(); - case capnp::DynamicValue::FLOAT: - return value.as(); - case capnp::DynamicValue::TEXT: - { - auto arr = value.as(); - return String(arr.begin(), arr.size()); - } - case capnp::DynamicValue::DATA: - { - auto arr = value.as().asChars(); - return String(arr.begin(), arr.size()); - } - case capnp::DynamicValue::LIST: - { - auto list_value = value.as(); - Array res(list_value.size()); - for (auto i : kj::indices(list_value)) - res[i] = convertNodeToField(list_value[i]); - - return res; - } - case capnp::DynamicValue::ENUM: - return value.as().getRaw(); - case capnp::DynamicValue::STRUCT: - { - auto struct_value = value.as(); - const auto & fields = struct_value.getSchema().getFields(); - - Tuple tuple(fields.size()); - for (auto i : kj::indices(fields)) - tuple[i] = convertNodeToField(struct_value.get(fields[i])); - - return tuple; - } - case capnp::DynamicValue::CAPABILITY: - throw Exception("CAPABILITY type not supported", ErrorCodes::BAD_TYPE_OF_FIELD); - case capnp::DynamicValue::ANY_POINTER: - throw Exception("ANY_POINTER type not supported", ErrorCodes::BAD_TYPE_OF_FIELD); - } - return Field(); -} - -static capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std::string & field) -{ - KJ_IF_MAYBE(child, node.findFieldByName(field)) - return *child; - else - throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr(), ErrorCodes::THERE_IS_NO_COLUMN); -} - - -void CapnProtoRowInputFormat::createActions(const NestedFieldList & sorted_fields, capnp::StructSchema reader) -{ - /// Columns in a table can map to fields in Cap'n'Proto or to structs. - - /// Store common parents and their tokens in order to backtrack. - std::vector parents; - std::vector parent_tokens; - - capnp::StructSchema cur_reader = reader; - - for (const auto & field : sorted_fields) - { - if (field.tokens.empty()) - throw Exception("Logical error in CapnProtoRowInputFormat", ErrorCodes::LOGICAL_ERROR); - - // Backtrack to common parent - while (field.tokens.size() < parent_tokens.size() + 1 - || !std::equal(parent_tokens.begin(), parent_tokens.end(), field.tokens.begin())) - { - actions.push_back({Action::POP}); - parents.pop_back(); - parent_tokens.pop_back(); - - if (parents.empty()) - { - cur_reader = reader; - break; - } - else - cur_reader = parents.back().getType().asStruct(); - } - - // Go forward - while (parent_tokens.size() + 1 < field.tokens.size()) - { - const auto & token = field.tokens[parents.size()]; - auto node = getFieldOrThrow(cur_reader, token); - if (node.getType().isStruct()) - { - // Descend to field structure - parents.emplace_back(node); - parent_tokens.emplace_back(token); - cur_reader = node.getType().asStruct(); - actions.push_back({Action::PUSH, node}); - } - else if (node.getType().isList()) - { - break; // Collect list - } - else - throw Exception("Field " + token + " is neither Struct nor List", ErrorCodes::BAD_TYPE_OF_FIELD); - } - - // Read field from the structure - auto node = getFieldOrThrow(cur_reader, field.tokens[parents.size()]); - if (node.getType().isList() && !actions.empty() && actions.back().field == node) - { - // The field list here flattens Nested elements into multiple arrays - // In order to map Nested types in Cap'nProto back, they need to be collected - // Since the field names are sorted, the order of field positions must be preserved - // For example, if the fields are { b @0 :Text, a @1 :Text }, the `a` would come first - // even though it's position is second. - auto & columns = actions.back().columns; - auto it = std::upper_bound(columns.cbegin(), columns.cend(), field.pos); - columns.insert(it, field.pos); - } - else - { - actions.push_back({Action::READ, node, {field.pos}}); - } - } -} - -CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info) - : IRowInputFormat(std::move(header), in_, std::move(params_)), parser(std::make_shared()) +CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_) + : IRowInputFormat(std::move(header), in_, std::move(params_)) + , parser(std::make_shared()) + , format_settings(format_settings_) + , column_types(getPort().getHeader().getDataTypes()) + , column_names(getPort().getHeader().getNames()) { // Parse the schema and fetch the root object - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wdeprecated-declarations" - auto schema = parser->impl.parseDiskFile(info.schemaPath(), info.absoluteSchemaPath(), {}); -#pragma GCC diagnostic pop - - root = schema.getNested(info.messageName()).asStruct(); - - /** - * The schema typically consists of fields in various nested structures. - * Here we gather the list of fields and sort them in a way so that fields in the same structure are adjacent, - * and the nesting level doesn't decrease to make traversal easier. - */ - const auto & sample = getPort().getHeader(); - NestedFieldList list; - size_t num_columns = sample.columns(); - for (size_t i = 0; i < num_columns; ++i) - list.push_back(split(sample, i)); - - // Order list first by value of strings then by length of string vector. - std::sort(list.begin(), list.end(), [](const NestedField & a, const NestedField & b) { return a.tokens < b.tokens; }); - createActions(list, root); + root = parser->getMessageSchema(info); + checkCapnProtoSchemaStructure(root, getPort().getHeader(), format_settings.capn_proto.enum_comparing_mode); } kj::Array CapnProtoRowInputFormat::readMessage() @@ -233,6 +74,186 @@ kj::Array CapnProtoRowInputFormat::readMessage() return msg; } +static void insertSignedInteger(IColumn & column, const DataTypePtr & column_type, Int64 value) +{ + switch (column_type->getTypeId()) + { + case TypeIndex::Int8: + assert_cast(column).insertValue(value); + break; + case TypeIndex::Int16: + assert_cast(column).insertValue(value); + break; + case TypeIndex::Int32: + assert_cast(column).insertValue(value); + break; + case TypeIndex::Int64: + assert_cast(column).insertValue(value); + break; + case TypeIndex::DateTime64: + assert_cast &>(column).insertValue(value); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not a signed integer."); + } +} + +static void insertUnsignedInteger(IColumn & column, const DataTypePtr & column_type, UInt64 value) +{ + switch (column_type->getTypeId()) + { + case TypeIndex::UInt8: + assert_cast(column).insertValue(value); + break; + case TypeIndex::Date: [[fallthrough]]; + case TypeIndex::UInt16: + assert_cast(column).insertValue(value); + break; + case TypeIndex::DateTime: [[fallthrough]]; + case TypeIndex::UInt32: + assert_cast(column).insertValue(value); + break; + case TypeIndex::UInt64: + assert_cast(column).insertValue(value); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not an unsigned integer."); + } +} + +static void insertFloat(IColumn & column, const DataTypePtr & column_type, Float64 value) +{ + switch (column_type->getTypeId()) + { + case TypeIndex::Float32: + assert_cast(column).insertValue(value); + break; + case TypeIndex::Float64: + assert_cast(column).insertValue(value); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Column type is not a float."); + } +} + +template +static void insertString(IColumn & column, Value value) +{ + column.insertData(reinterpret_cast(value.begin()), value.size()); +} + +template +static void insertEnum(IColumn & column, const DataTypePtr & column_type, const capnp::DynamicEnum & enum_value, FormatSettings::EnumComparingMode enum_comparing_mode) +{ + auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); + auto enum_type = assert_cast *>(column_type.get()); + DataTypePtr nested_type = std::make_shared>(); + if (enum_comparing_mode == FormatSettings::EnumComparingMode::BY_VALUES) + insertSignedInteger(column, nested_type, Int64(enumerant.getOrdinal())); + else if (enum_comparing_mode == FormatSettings::EnumComparingMode::BY_NAMES) + insertSignedInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName())))); + else + { + /// Find the same enum name case insensitive. + String enum_name = enumerant.getProto().getName(); + for (auto & name : enum_type->getAllRegisteredNames()) + { + if (compareEnumNames(name, enum_name, enum_comparing_mode)) + { + insertSignedInteger(column, nested_type, Int64(enum_type->getValue(name))); + break; + } + } + } +} + +static void insertValue(IColumn & column, const DataTypePtr & column_type, const capnp::DynamicValue::Reader & value, FormatSettings::EnumComparingMode enum_comparing_mode) +{ + if (column_type->lowCardinality()) + { + auto & lc_column = assert_cast(column); + auto tmp_column = lc_column.getDictionary().getNestedColumn()->cloneEmpty(); + auto dict_type = assert_cast(column_type.get())->getDictionaryType(); + insertValue(*tmp_column, dict_type, value, enum_comparing_mode); + lc_column.insertFromFullColumn(*tmp_column, 0); + return; + } + + switch (value.getType()) + { + case capnp::DynamicValue::Type::INT: + insertSignedInteger(column, column_type, value.as()); + break; + case capnp::DynamicValue::Type::UINT: + insertUnsignedInteger(column, column_type, value.as()); + break; + case capnp::DynamicValue::Type::FLOAT: + insertFloat(column, column_type, value.as()); + break; + case capnp::DynamicValue::Type::BOOL: + insertUnsignedInteger(column, column_type, UInt64(value.as())); + break; + case capnp::DynamicValue::Type::DATA: + insertString(column, value.as()); + break; + case capnp::DynamicValue::Type::TEXT: + insertString(column, value.as()); + break; + case capnp::DynamicValue::Type::ENUM: + if (column_type->getTypeId() == TypeIndex::Enum8) + insertEnum(column, column_type, value.as(), enum_comparing_mode); + else + insertEnum(column, column_type, value.as(), enum_comparing_mode); + break; + case capnp::DynamicValue::LIST: + { + auto list_value = value.as(); + auto & column_array = assert_cast(column); + auto & offsets = column_array.getOffsets(); + offsets.push_back(offsets.back() + list_value.size()); + + auto & nested_column = column_array.getData(); + auto nested_type = assert_cast(column_type.get())->getNestedType(); + for (const auto & nested_value : list_value) + insertValue(nested_column, nested_type, nested_value, enum_comparing_mode); + break; + } + case capnp::DynamicValue::Type::STRUCT: + { + auto struct_value = value.as(); + if (column_type->isNullable()) + { + auto & nullable_column = assert_cast(column); + auto field = *kj::_::readMaybe(struct_value.which()); + if (field.getType().isVoid()) + nullable_column.insertDefault(); + else + { + auto & nested_column = nullable_column.getNestedColumn(); + auto nested_type = assert_cast(column_type.get())->getNestedType(); + auto nested_value = struct_value.get(field); + insertValue(nested_column, nested_type, nested_value, enum_comparing_mode); + nullable_column.getNullMapData().push_back(0); + } + } + else + { + auto & tuple_column = assert_cast(column); + const auto * tuple_type = assert_cast(column_type.get()); + for (size_t i = 0; i != tuple_column.tupleSize(); ++i) + insertValue( + tuple_column.getColumn(i), + tuple_type->getElements()[i], + struct_value.get(tuple_type->getElementNames()[i]), + enum_comparing_mode); + } + break; + } + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto value type."); + } +} + bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { if (in->eof()) @@ -245,51 +266,12 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension #else capnp::FlatArrayMessageReader msg(array); #endif - std::vector stack; - stack.push_back(msg.getRoot(root)); - for (auto action : actions) + auto root_reader = msg.getRoot(root); + for (size_t i = 0; i != columns.size(); ++i) { - switch (action.type) - { - case Action::READ: - { - Field value = convertNodeToField(stack.back().get(action.field)); - if (action.columns.size() > 1) - { - // Nested columns must be flattened into several arrays - // e.g. Array(Tuple(x ..., y ...)) -> Array(x ...), Array(y ...) - const auto & collected = DB::get(value); - size_t size = collected.size(); - // The flattened array contains an array of a part of the nested tuple - Array flattened(size); - for (size_t column_index = 0; column_index < action.columns.size(); ++column_index) - { - // Populate array with a single tuple elements - for (size_t off = 0; off < size; ++off) - { - const auto & tuple = DB::get(collected[off]); - flattened[off] = tuple[column_index]; - } - auto & col = columns[action.columns[column_index]]; - col->insert(flattened); - } - } - else - { - auto & col = columns[action.columns[0]]; - col->insert(value); - } - - break; - } - case Action::POP: - stack.pop_back(); - break; - case Action::PUSH: - stack.push_back(stack.back().get(action.field).as()); - break; - } + auto value = getReaderByColumnName(root_reader, column_names[i]); + insertValue(*columns[i], column_types[i], value, format_settings.capn_proto.enum_comparing_mode); } return true; @@ -302,8 +284,7 @@ void registerInputFormatCapnProto(FormatFactory & factory) [](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(settings.schema.format_schema, "CapnProto", true, - settings.schema.is_server, settings.schema.format_schema_path)); + FormatSchemaInfo(settings, "CapnProto", true), settings); }); } diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index 0957cd1d681..fc30cf11237 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -4,8 +4,8 @@ #if USE_CAPNP #include +#include #include -#include namespace DB { @@ -22,18 +22,7 @@ class ReadBuffer; class CapnProtoRowInputFormat : public IRowInputFormat { public: - struct NestedField - { - std::vector tokens; - size_t pos; - }; - using NestedFieldList = std::vector; - - /** schema_dir - base path for schema files - * schema_file - location of the capnproto schema, e.g. "schema.capnp" - * root_object - name to the root object, e.g. "Message" - */ - CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info); + CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_); String getName() const override { return "CapnProtoRowInputFormat"; } @@ -42,34 +31,11 @@ public: private: kj::Array readMessage(); - // Build a traversal plan from a sorted list of fields - void createActions(const NestedFieldList & sorted_fields, capnp::StructSchema reader); - - /* Action for state machine for traversing nested structures. */ - using BlockPositionList = std::vector; - struct Action - { - enum Type { POP, PUSH, READ }; - Type type{}; - capnp::StructSchema::Field field{}; - BlockPositionList columns{}; - }; - - // Wrapper for classes that could throw in destructor - // https://github.com/capnproto/capnproto/issues/553 - template - struct DestructorCatcher - { - T impl; - template - DestructorCatcher(Arg && ... args) : impl(kj::fwd(args)...) {} - ~DestructorCatcher() noexcept try { } catch (...) { return; } - }; - using SchemaParser = DestructorCatcher; - - std::shared_ptr parser; + std::shared_ptr parser; capnp::StructSchema root; - std::vector actions; + const FormatSettings format_settings; + DataTypes column_types; + Names column_names; }; } diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp new file mode 100644 index 00000000000..b299e1fc00a --- /dev/null +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -0,0 +1,251 @@ +#include +#if USE_CAPNP + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + +CapnProtoOutputStream::CapnProtoOutputStream(WriteBuffer & out_) : out(out_) +{ +} + +void CapnProtoOutputStream::write(const void * buffer, size_t size) +{ + out.write(reinterpret_cast(buffer), size); +} + +CapnProtoRowOutputFormat::CapnProtoRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + const RowOutputFormatParams & params_, + const FormatSchemaInfo & info, + const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), column_names(header_.getNames()), column_types(header_.getDataTypes()), output_stream(std::make_unique(out_)), format_settings(format_settings_) +{ + schema = schema_parser.getMessageSchema(info); + checkCapnProtoSchemaStructure(schema, getPort(PortKind::Main).getHeader(), format_settings.capn_proto.enum_comparing_mode); +} + +template +static capnp::DynamicEnum getDynamicEnum( + const ColumnPtr & column, + const DataTypePtr & data_type, + size_t row_num, + const capnp::EnumSchema & enum_schema, + FormatSettings::EnumComparingMode mode) +{ + const auto * enum_data_type = assert_cast *>(data_type.get()); + EnumValue enum_value = column->getInt(row_num); + if (mode == FormatSettings::EnumComparingMode::BY_VALUES) + return capnp::DynamicEnum(enum_schema, enum_value); + + auto enum_name = enum_data_type->getNameForValue(enum_value); + for (const auto enumerant : enum_schema.getEnumerants()) + { + if (compareEnumNames(String(enum_name), enumerant.getProto().getName(), mode)) + return capnp::DynamicEnum(enumerant); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert CLickHouse Enum value to CapnProto Enum"); +} + +static capnp::DynamicValue::Builder initStructFieldBuilder(const ColumnPtr & column, size_t row_num, capnp::DynamicStruct::Builder & struct_builder, capnp::StructSchema::Field field) +{ + if (const auto * array_column = checkAndGetColumn(*column)) + { + size_t size = array_column->getOffsets()[row_num] - array_column->getOffsets()[row_num - 1]; + return struct_builder.init(field, size); + } + + if (field.getType().isStruct()) + return struct_builder.init(field); + + return struct_builder.get(field); +} + +static std::optional convertToDynamicValue(const ColumnPtr & column, const DataTypePtr & data_type, size_t row_num, capnp::DynamicValue::Builder builder, FormatSettings::EnumComparingMode enum_comparing_mode) +{ + /// Here we don't do any types validation, because we did it in CapnProtoRowOutputFormat constructor. + + if (data_type->lowCardinality()) + { + const auto * lc_column = assert_cast(column.get()); + const auto & dict_type = assert_cast(data_type.get())->getDictionaryType(); + size_t index = lc_column->getIndexAt(row_num); + return convertToDynamicValue(lc_column->getDictionary().getNestedColumn(), dict_type, index, builder, enum_comparing_mode); + } + + switch (builder.getType()) + { + case capnp::DynamicValue::Type::INT: + /// We allow output DateTime64 as Int64. + if (WhichDataType(data_type).isDateTime64()) + return capnp::DynamicValue::Reader(assert_cast *>(column.get())->getElement(row_num)); + return capnp::DynamicValue::Reader(column->getInt(row_num)); + case capnp::DynamicValue::Type::UINT: + return capnp::DynamicValue::Reader(column->getUInt(row_num)); + case capnp::DynamicValue::Type::BOOL: + return capnp::DynamicValue::Reader(column->getBool(row_num)); + case capnp::DynamicValue::Type::FLOAT: + return capnp::DynamicValue::Reader(column->getFloat64(row_num)); + case capnp::DynamicValue::Type::ENUM: + { + auto enum_schema = builder.as().getSchema(); + if (data_type->getTypeId() == TypeIndex::Enum8) + return capnp::DynamicValue::Reader( + getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); + return capnp::DynamicValue::Reader( + getDynamicEnum(column, data_type, row_num, enum_schema, enum_comparing_mode)); + } + case capnp::DynamicValue::Type::DATA: + { + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Data::Reader(reinterpret_cast(data.data), data.size)); + } + case capnp::DynamicValue::Type::TEXT: + { + auto data = String(column->getDataAt(row_num)); + return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data(), data.size())); + } + case capnp::DynamicValue::Type::STRUCT: + { + auto struct_builder = builder.as(); + auto nested_struct_schema = struct_builder.getSchema(); + /// Struct can be represent Tuple or Naullable (named union with two fields) + if (data_type->isNullable()) + { + const auto * nullable_type = assert_cast(data_type.get()); + const auto * nullable_column = assert_cast(column.get()); + auto fields = nested_struct_schema.getUnionFields(); + if (nullable_column->isNullAt(row_num)) + { + auto null_field = fields[0].getType().isVoid() ? fields[0] : fields[1]; + struct_builder.set(null_field, capnp::Void()); + } + else + { + auto value_field = fields[0].getType().isVoid() ? fields[1] : fields[0]; + struct_builder.clear(value_field); + const auto & nested_column = nullable_column->getNestedColumnPtr(); + auto value_builder = initStructFieldBuilder(nested_column, row_num, struct_builder, value_field); + auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, value_builder, enum_comparing_mode); + if (value) + struct_builder.set(value_field, std::move(*value)); + } + } + else + { + const auto * tuple_data_type = assert_cast(data_type.get()); + auto nested_types = tuple_data_type->getElements(); + const auto & nested_columns = assert_cast(column.get())->getColumns(); + for (const auto & name : tuple_data_type->getElementNames()) + { + auto pos = tuple_data_type->getPositionByName(name); + auto field_builder + = initStructFieldBuilder(nested_columns[pos], row_num, struct_builder, nested_struct_schema.getFieldByName(name)); + auto value = convertToDynamicValue(nested_columns[pos], nested_types[pos], row_num, field_builder, enum_comparing_mode); + if (value) + struct_builder.set(name, std::move(*value)); + } + } + return std::nullopt; + } + case capnp::DynamicValue::Type::LIST: + { + auto list_builder = builder.as(); + const auto * array_column = assert_cast(column.get()); + const auto & nested_column = array_column->getDataPtr(); + const auto & nested_type = assert_cast(data_type.get())->getNestedType(); + const auto & offsets = array_column->getOffsets(); + auto offset = offsets[row_num - 1]; + size_t size = offsets[row_num] - offset; + + const auto * nested_array_column = checkAndGetColumn(*nested_column); + for (size_t i = 0; i != size; ++i) + { + capnp::DynamicValue::Builder value_builder; + /// For nested arrays we need to initialize nested list builder. + if (nested_array_column) + { + const auto & nested_offset = nested_array_column->getOffsets(); + size_t nested_array_size = nested_offset[offset + i] - nested_offset[offset + i - 1]; + value_builder = list_builder.init(i, nested_array_size); + } + else + value_builder = list_builder[i]; + + auto value = convertToDynamicValue(nested_column, nested_type, offset + i, value_builder, enum_comparing_mode); + if (value) + list_builder.set(i, std::move(*value)); + } + return std::nullopt; + } + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected CapnProto type."); + } +} + +void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) +{ + capnp::MallocMessageBuilder message; + capnp::DynamicStruct::Builder root = message.initRoot(schema); + for (size_t i = 0; i != columns.size(); ++i) + { + auto [struct_builder, field] = getStructBuilderAndFieldByColumnName(root, column_names[i]); + auto field_builder = initStructFieldBuilder(columns[i], row_num, struct_builder, field); + auto value = convertToDynamicValue(columns[i], column_types[i], row_num, field_builder, format_settings.capn_proto.enum_comparing_mode); + if (value) + struct_builder.set(field, std::move(*value)); + } + + capnp::writeMessage(*output_stream, message); +} + +void registerOutputFormatProcessorsCapnProto(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("CapnProto", []( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, params, FormatSchemaInfo(format_settings, "CapnProto", true), format_settings); + }); +} + +} + +#else + +namespace DB +{ +class FormatFactory; +void registerOutputFormatProcessorsCapnProto(FormatFactory &) {} +} + +#endif // USE_CAPNP diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h new file mode 100644 index 00000000000..0f321071d62 --- /dev/null +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -0,0 +1,53 @@ +#pragma once + +#include "config_formats.h" +#if USE_CAPNP + +#include +#include +#include +#include +#include +#include + +namespace DB +{ +class CapnProtoOutputStream : public kj::OutputStream +{ +public: + CapnProtoOutputStream(WriteBuffer & out_); + + void write(const void * buffer, size_t size) override; + +private: + WriteBuffer & out; +}; + +class CapnProtoRowOutputFormat : public IRowOutputFormat +{ +public: + CapnProtoRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + const RowOutputFormatParams & params_, + const FormatSchemaInfo & info, + const FormatSettings & format_settings_); + + String getName() const override { return "CapnProtoRowOutputFormat"; } + + void write(const Columns & columns, size_t row_num) override; + + void writeField(const IColumn &, const ISerialization &, size_t) override { } + +private: + Names column_names; + DataTypes column_types; + capnp::StructSchema schema; + std::unique_ptr output_stream; + const FormatSettings format_settings; + CapnProtoSchemaParser schema_parser; +}; + +} + +#endif // USE_CAPNP diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index a5e6b7ec480..df7b7102739 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -67,8 +67,7 @@ void registerInputFormatProtobuf(FormatFactory & factory) const FormatSettings & settings) { return std::make_shared(buf, sample, std::move(params), - FormatSchemaInfo(settings.schema.format_schema, "Protobuf", true, - settings.schema.is_server, settings.schema.format_schema_path), + FormatSchemaInfo(settings, "Protobuf", true), with_length_delimiter); }); } diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp index 12c5e98797a..29cd9be79bc 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.cpp @@ -64,9 +64,7 @@ void registerOutputFormatProtobuf(FormatFactory & factory) { return std::make_shared( buf, header, params, - FormatSchemaInfo(settings.schema.format_schema, "Protobuf", - true, settings.schema.is_server, - settings.schema.format_schema_path), + FormatSchemaInfo(settings, "Protobuf", true), settings, with_length_delimiter); }); diff --git a/tests/queries/0_stateless/02030_capnp_format.reference b/tests/queries/0_stateless/02030_capnp_format.reference new file mode 100644 index 00000000000..2b2307bfc6a --- /dev/null +++ b/tests/queries/0_stateless/02030_capnp_format.reference @@ -0,0 +1,52 @@ +-1 1 -1000 1000 -10000000 1000000 -1000000000 1000000000 123.123 123123123.12312312 Some string fixed Some data 2000-01-06 2000-06-01 19:42:42 2000-04-01 11:21:33.123 +-1 1 -1000 1000 -10000000 1000000 -1000000000 1000000000 123.123 123123123.12312312 Some string fixed Some data 2000-01-06 2000-06-01 19:42:42 2000-04-01 11:21:33.123 +1 (2,(3,4)) (((5))) +1 (2,(3,4)) (((5))) +1 [1,2,3] [[[1,2,3],[4,5,6]],[[7,8,9],[]],[]] +1 [1,2,3] [[[1,2,3],[4,5,6]],[[7,8,9],[]],[]] +1 ((2,[[3,4],[5,6],[]]),[([[(7,8),(9,10)],[(11,12),(13,14)],[]],[([15,16,17]),([])])]) +1 ((2,[[3,4],[5,6],[]]),[([[(7,8),(9,10)],[(11,12),(13,14)],[]],[([15,16,17]),([])])]) +[1,2,3] [[4,5,6],[],[7,8]] [(9,10),(11,12),(13,14)] +[1,2,3] [[4,5,6],[],[7,8]] [(9,10),(11,12),(13,14)] +1 [1,NULL,2] (1) +\N [NULL,NULL,42] (NULL) +1 [1,NULL,2] (1) +\N [NULL,NULL,42] (NULL) +one +two +tHrEe +oNe +tWo +threE +first +second +third +OK +OK +OK +OK +one two ['one',NULL,'two',NULL] +two \N [NULL] +one two ['one',NULL,'two',NULL] +two \N [NULL] +0 1 2 +1 2 3 +2 3 4 +3 4 5 +4 5 6 +(0,(1,(2))) +(1,(2,(3))) +(2,(3,(4))) +(3,(4,(5))) +(4,(5,(6))) +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK +OK diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh new file mode 100755 index 00000000000..99807cc1738 --- /dev/null +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -0,0 +1,109 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +CAPN_PROTO_FILE=$USER_FILES_PATH/data.capnp +touch $CAPN_PROTO_FILE + +SCHEMADIR=/$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +CLIENT_SCHEMADIR=$CURDIR/format_schemas +SERVER_SCHEMADIR=test_02030 +mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR +cp -r $CLIENT_SCHEMADIR/02030_* $SCHEMADIR/$SERVER_SCHEMADIR/ + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_simple_types"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_simple_types (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixed FixedString(5), data String, date Date, datetime DateTime, datetime64 DateTime64(3)) ENGINE=Memory" +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types values (-1, 1, -1000, 1000, -10000000, 1000000, -1000000000, 1000000000, 123.123, 123123123.123123123, 'Some string', 'fixed', 'Some data', '2000-01-06', '2000-06-01 19:42:42', '2000-04-01 11:21:33.123')" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_simple_types FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_simple_types" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_simple_types" + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_tuples" +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_tuples (value UInt64, tuple1 Tuple(one UInt64, two Tuple(three UInt64, four UInt64)), tuple2 Tuple(nested1 Tuple(nested2 Tuple(x UInt64)))) ENGINE=Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples VALUES (1, (2, (3, 4)), (((5))))" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_tuples:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_tuples" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_tuples" + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_lists" +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_lists (value UInt64, list1 Array(UInt64), list2 Array(Array(Array(UInt64)))) ENGINE=Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists VALUES (1, [1, 2, 3], [[[1, 2, 3], [4, 5, 6]], [[7, 8, 9], []], []])" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_lists FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_lists:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_lists" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_lists" + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nested_lists_and_tuples" +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nested_lists_and_tuples (value UInt64, nested Tuple(a Tuple(b UInt64, c Array(Array(UInt64))), d Array(Tuple(e Array(Array(Tuple(f UInt64, g UInt64))), h Array(Tuple(k Array(UInt64))))))) ENGINE=Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples VALUES (1, ((2, [[3, 4], [5, 6], []]), [([[(7, 8), (9, 10)], [(11, 12), (13, 14)], []], [([15, 16, 17]), ([])])]))" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_lists_and_tuples FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_lists_and_tuples:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_lists_and_tuples" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_lists_and_tuples" + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nested_table" +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nested_table (nested Nested(value UInt64, array Array(UInt64), tuple Tuple(one UInt64, two UInt64))) ENGINE=Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table VALUES ([1, 2, 3], [[4, 5, 6], [], [7, 8]], [(9, 10), (11, 12), (13, 14)])" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nested_table FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_table:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nested_table" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nested_table" + + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_nullable" +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_nullable (nullable Nullable(UInt64), array Array(Nullable(UInt64)), tuple Tuple(nullable Nullable(UInt64))) ENGINE=Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable VALUES (1, [1, Null, 2], (1)), (Null, [Null, Null, 42], (Null))" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_nullable FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nullable:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_nullable" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_nullable" + + +$CLICKHOUSE_CLIENT --query="SELECT CAST(number, 'Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2)') AS value FROM numbers(3) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_enum:Message'" > $CAPN_PROTO_FILE + +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 1, \'two\' = 2, \'tHrEe\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'oNe\' = 1, \'tWo\' = 2, \'threE\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names_case_insensitive'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 0, \'second\' = 1, \'third\' = 2)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 0, \'two\' = 1, \'three\' = 2)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 0, \'two\' = 1, \'tHrEe\' = 2, \'four\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'one\' = 1, \'two\' = 2, \'tHrEe\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_values'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'value Enum(\'first\' = 1, \'two\' = 2, \'three\' = 3)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_enum:Message', format_capn_proto_enum_comparising_mode='by_names_case_insensitive'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS capnp_low_cardinality" +$CLICKHOUSE_CLIENT --query="CREATE TABLE capnp_low_cardinality (lc1 LowCardinality(String), lc2 LowCardinality(Nullable(String)), lc3 Array(LowCardinality(Nullable(String)))) ENGINE=Memory" +$CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality VALUES ('one', 'two', ['one', Null, 'two', Null]), ('two', Null, [Null])" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" | $CLICKHOUSE_CLIENT --query="INSERT INTO capnp_low_cardinality FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_low_cardinality:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM capnp_low_cardinality" +$CLICKHOUSE_CLIENT --query="DROP TABLE capnp_low_cardinality" + + +$CLICKHOUSE_CLIENT --query="SELECT CAST(tuple(number, tuple(number + 1, tuple(number + 2))), 'Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') AS a FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_tuples:Message'" > $CAPN_PROTO_FILE +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a_b UInt64, a_c_d UInt64, a_c_e_f UInt64') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" + + +$CLICKHOUSE_CLIENT --query="SELECT number AS a_b, number + 1 AS a_c_d, number + 2 AS a_c_e_f FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_nested_tuples:Message'" > $CAPN_PROTO_FILE +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(bb UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(ff UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; + +$CLICKHOUSE_CLIENT --query="SELECT number AS uint64 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" > $CAPN_PROTO_FILE +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 String') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Array(UInt64)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Enum(\'one\' = 1)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Tuple(UInt64)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Nullable(UInt64)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Int32') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; + + +$CLICKHOUSE_CLIENT --query="SELECT number AS a, toString(number) as b FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_unnamed_union:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable1 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_fake_nullable:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; +$CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable2 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_fake_nullable:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; + +rm $CAPN_PROTO_FILE +rm -rf $SCHEMADIR/$SERVER_SCHEMADIR diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_enum.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_enum.capnp new file mode 100644 index 00000000000..f033b177a45 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_enum.capnp @@ -0,0 +1,13 @@ +@0x9ef128e10a8010b2; + +struct Message +{ + value @0 : EnumType; + + enum EnumType + { + one @0; + two @1; + tHrEe @2; + } +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_fake_nullable.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_fake_nullable.capnp new file mode 100644 index 00000000000..a027692e4bc --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_fake_nullable.capnp @@ -0,0 +1,23 @@ +@0xd8dd7b35452d1c4c; + +struct FakeNullable1 +{ + union + { + value @0 : Text; + null @1 : Void; + trash @2 : Text; + } +} + +struct FakeNullable2 +{ + value @0 : Text; + null @1 : Void; +} + +struct Message +{ + nullable1 @0 : FakeNullable1; + nullable2 @1 : FakeNullable2; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_lists.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_lists.capnp new file mode 100644 index 00000000000..78fe3cf551e --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_lists.capnp @@ -0,0 +1,8 @@ +@0x9ef128e10a8010b7; + +struct Message +{ + value @0 : UInt64; + list1 @1 : List(UInt64); + list2 @2 : List(List(List(UInt64))); +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_low_cardinality.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_low_cardinality.capnp new file mode 100644 index 00000000000..0958889f0d8 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_low_cardinality.capnp @@ -0,0 +1,17 @@ +@0x9ef128e10a8010b7; + +struct NullableText +{ + union + { + value @0 : Text; + null @1 : Void; + } +} + +struct Message +{ + lc1 @0 : Text; + lc2 @1 : NullableText; + lc3 @2 : List(NullableText); +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_nested_lists_and_tuples.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_nested_lists_and_tuples.capnp new file mode 100644 index 00000000000..11fa99f62f5 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_nested_lists_and_tuples.capnp @@ -0,0 +1,36 @@ +@0x9ef128e10a8010b2; + +struct Nested1 +{ + b @0 : UInt64; + c @1 : List(List(UInt64)); +} + +struct Nested2 +{ + e @0 : List(List(Nested3)); + h @1 : List(Nested4); +} + +struct Nested3 +{ + f @0 : UInt64; + g @1 : UInt64; +} + +struct Nested4 +{ + k @0 : List(UInt64); +} + +struct Nested +{ + a @0 : Nested1; + d @1 : List(Nested2); +} + +struct Message +{ + value @0 : UInt64; + nested @1 : Nested; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_nested_table.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_nested_table.capnp new file mode 100644 index 00000000000..42f17246d58 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_nested_table.capnp @@ -0,0 +1,20 @@ +@0x9ef128e10a8010b3; + + +struct Nested1 +{ + one @0 : UInt64; + two @1 : UInt64; +} + +struct Nested +{ + value @0 : List(UInt64); + array @1 : List(List(UInt64)); + tuple @2 : List(Nested1); +} + +struct Message +{ + nested @0 : Nested; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_nested_tuples.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_nested_tuples.capnp new file mode 100644 index 00000000000..161c1bbaea6 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_nested_tuples.capnp @@ -0,0 +1,23 @@ +@0x9ef128e12a8010b2; + +struct Nested1 +{ + d @0 : UInt64; + e @1 : Nested2; +} + +struct Nested2 +{ + f @0 : UInt64; +} + +struct Nested +{ + b @0 : UInt64; + c @1 : Nested1; +} + +struct Message +{ + a @0 : Nested; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_nullable.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_nullable.capnp new file mode 100644 index 00000000000..41254911710 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_nullable.capnp @@ -0,0 +1,22 @@ +@0x9ef128e10a8010b2; + +struct NullableUInt64 +{ + union + { + value @0 : UInt64; + null @1 : Void; + } +} + +struct Tuple +{ + nullable @0 : NullableUInt64; +} + +struct Message +{ + nullable @0 : NullableUInt64; + array @1 : List(NullableUInt64); + tuple @2 : Tuple; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_simple_types.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_simple_types.capnp new file mode 100644 index 00000000000..a85bbbc511b --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_simple_types.capnp @@ -0,0 +1,21 @@ +@0xd9dd7b35452d1c4f; + +struct Message +{ + int8 @0 : Int8; + uint8 @1 : UInt8; + int16 @2 : Int16; + uint16 @3 : UInt16; + int32 @4 : Int32; + uint32 @5 : UInt32; + int64 @6 : Int64; + uint64 @7 : UInt64; + float32 @8 : Float32; + float64 @9 : Float64; + string @10 : Text; + fixed @11 : Text; + data @12 : Data; + date @13 : UInt16; + datetime @14 : UInt32; + datetime64 @15 : Int64; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_tuples.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_tuples.capnp new file mode 100644 index 00000000000..21c3f0eb2e1 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_tuples.capnp @@ -0,0 +1,35 @@ +@0x9ef128e10a8010b8; + +struct Nested5 +{ + x @0 : UInt64; +} + +struct Nested4 +{ + nested2 @0 : Nested5; +} + +struct Nested3 +{ + nested1 @0 : Nested4; +} + +struct Nested2 +{ + three @0 : UInt64; + four @1 : UInt64; +} + +struct Nested1 +{ + one @0 : UInt64; + two @1 : Nested2; +} + +struct Message +{ + value @0 : UInt64; + tuple1 @1 : Nested1; + tuple2 @2 : Nested3; +} diff --git a/tests/queries/0_stateless/format_schemas/02030_capnp_unnamed_union.capnp b/tests/queries/0_stateless/format_schemas/02030_capnp_unnamed_union.capnp new file mode 100644 index 00000000000..9fb5e37bfea --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/02030_capnp_unnamed_union.capnp @@ -0,0 +1,10 @@ +@0xd8dd7b35452d1c4f; + +struct Message +{ + union + { + a @0 : UInt64; + b @1 : Text; + } +} From 1cd938fbba61053e5a2d77b53afa14d7a35436ce Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 28 Sep 2021 16:07:00 +0300 Subject: [PATCH 647/950] Fix typo --- src/Formats/FormatSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index ce5f1effa8c..403ccbc6763 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -184,7 +184,7 @@ struct FormatSettings bool import_nested = false; } orc; - /// For apnProto format we should determine how to + /// For capnProto format we should determine how to /// compare ClickHouse Enum and Enum from schema. enum class EnumComparingMode { From c97f375728eb372ddc50a927372685bce7e5226a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 28 Sep 2021 17:51:10 +0300 Subject: [PATCH 648/950] Fix style --- tests/queries/0_stateless/02030_capnp_format.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 99807cc1738..03b43c007d8 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -12,7 +12,7 @@ touch $CAPN_PROTO_FILE SCHEMADIR=/$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02030 -mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR +mkdir -p ${SCHEMADIR:?}/${SERVER_SCHEMADIR:?} cp -r $CLIENT_SCHEMADIR/02030_* $SCHEMADIR/$SERVER_SCHEMADIR/ From ed8818a773a82dc47ca4bb88e565267c8c954dcb Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 28 Sep 2021 20:03:03 +0300 Subject: [PATCH 649/950] Fix style, better check in enum comparison --- src/Formats/CapnProtoUtils.cpp | 12 +++++++----- tests/queries/0_stateless/02030_capnp_format.sh | 2 +- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 9931785f43e..974688e7560 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -153,16 +153,18 @@ static bool checkEnums(const capnp::Type & capnp_type, const DataTypePtr column_ std::unordered_set capn_enum_values; auto enumerants = enum_schema.getEnumerants(); + /// In CapnProto Enum fields are numbered sequentially starting from zero. + if (mode == FormatSettings::EnumComparingMode::BY_VALUES && enumerants.size() > max_value) + { + error_message += "Enum from CapnProto schema contains values that is out of range for Clickhouse Enum"; + return false; + } + for (auto enumerant : enumerants) { String name = enumerant.getProto().getName(); capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); auto value = enumerant.getOrdinal(); - if (mode == FormatSettings::EnumComparingMode::BY_VALUES && value > max_value) - { - error_message += "Enum from CapnProto schema contains value that is out of range for Clickhouse Enum"; - return false; - } capn_enum_values.insert(Type(value)); } diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 03b43c007d8..1a0efe4ed07 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -106,4 +106,4 @@ $CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable1 FRO $CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable2 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_fake_nullable:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; rm $CAPN_PROTO_FILE -rm -rf $SCHEMADIR/$SERVER_SCHEMADIR +rm -rf {$SCHEMADIR:?}/{$SERVER_SCHEMADIR:?} From 17ed293470d65738a0404ea53cff6cbda58b5a61 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 29 Sep 2021 14:21:20 +0300 Subject: [PATCH 650/950] Fix test --- tests/queries/0_stateless/02030_capnp_format.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 1a0efe4ed07..e6592142560 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -12,7 +12,7 @@ touch $CAPN_PROTO_FILE SCHEMADIR=/$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02030 -mkdir -p ${SCHEMADIR:?}/${SERVER_SCHEMADIR:?} +mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR cp -r $CLIENT_SCHEMADIR/02030_* $SCHEMADIR/$SERVER_SCHEMADIR/ @@ -106,4 +106,4 @@ $CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable1 FRO $CLICKHOUSE_CLIENT --query="SELECT toNullable(toString(number)) as nullable2 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_fake_nullable:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; rm $CAPN_PROTO_FILE -rm -rf {$SCHEMADIR:?}/{$SERVER_SCHEMADIR:?} +rm -rf ${SCHEMADIR:?}/${SERVER_SCHEMADIR:?} From f88a2ad653f4a5ed2f0dc5a9d008020e91b0a09a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 29 Sep 2021 15:08:53 +0300 Subject: [PATCH 651/950] Handle exception when cannot extract value from struct, add test for it --- src/Formats/CapnProtoUtils.cpp | 11 ++++++++++- .../queries/0_stateless/02030_capnp_format.reference | 1 + tests/queries/0_stateless/02030_capnp_format.sh | 4 ++++ 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 974688e7560..9176579f672 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -331,7 +331,16 @@ capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Re auto [field_name, nested_name] = splitFieldName(name); KJ_IF_MAYBE(field, struct_reader.getSchema().findFieldByName(field_name)) { - auto field_reader = struct_reader.get(*field); + capnp::DynamicValue::Reader field_reader; + try + { + field_reader = struct_reader.get(*field); + } + catch (const kj::Exception & e) + { + throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot extract field value from struct by provided schema, error: {} Perhaps the data was generated by another schema", String(e.getDescription().cStr())); + } + if (nested_name.empty()) return field_reader; diff --git a/tests/queries/0_stateless/02030_capnp_format.reference b/tests/queries/0_stateless/02030_capnp_format.reference index 2b2307bfc6a..8c3c81b5bc3 100644 --- a/tests/queries/0_stateless/02030_capnp_format.reference +++ b/tests/queries/0_stateless/02030_capnp_format.reference @@ -50,3 +50,4 @@ OK OK OK OK +OK diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index e6592142560..c24b85109da 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -92,6 +92,10 @@ $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tup $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(bb UInt64, c Tuple(d UInt64, e Tuple(f UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'a Tuple(b UInt64, c Tuple(d UInt64, e Tuple(ff UInt64)))') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_nested_tuples:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'string String') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'; + + $CLICKHOUSE_CLIENT --query="SELECT number AS uint64 FROM numbers(5) FORMAT CapnProto SETTINGS format_schema='$CLIENT_SCHEMADIR/02030_capnp_simple_types:Message'" > $CAPN_PROTO_FILE $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 String') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT --query="SELECT * FROM file('data.capnp', 'CapnProto', 'uint64 Array(UInt64)') SETTINGS format_schema='$SERVER_SCHEMADIR/02030_capnp_simple_types:Message'" 2>&1 | grep -F -q "CAPN_PROTO_BAD_CAST" && echo 'OK' || echo 'FAIL'; From 9ddcdbba39bda24408874207762f8ffb669058df Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 30 Sep 2021 23:19:21 +0300 Subject: [PATCH 652/950] Add INCORRECT_DATA error code --- src/Formats/CapnProtoUtils.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 9176579f672..2cc20abedd0 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int CAPN_PROTO_BAD_CAST; extern const int FILE_DOESNT_EXIST; extern const int UNKNOWN_EXCEPTION; + extern const int INCORRECT_DATA; } capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info) From 5d16dc7f9aa82b9952578e6672cc9ab84bd5f0d4 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 4 Oct 2021 16:02:32 +0300 Subject: [PATCH 653/950] Try to fix tests, update capnp lib to eliminate problem with UB sanitizer --- contrib/capnproto | 2 +- contrib/capnproto-cmake/CMakeLists.txt | 1 + src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp | 6 +++--- tests/queries/0_stateless/02030_capnp_format.sh | 4 ++-- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/contrib/capnproto b/contrib/capnproto index a00ccd91b37..c8189ec3c27 160000 --- a/contrib/capnproto +++ b/contrib/capnproto @@ -1 +1 @@ -Subproject commit a00ccd91b3746ef2ab51d40fe3265829949d1ace +Subproject commit c8189ec3c27dacbd4a3288e682473010e377f593 diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index 9f6e076cc7d..274be8c5eeb 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -45,6 +45,7 @@ set (CAPNP_SRCS "${CAPNPROTO_SOURCE_DIR}/capnp/serialize-packed.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/schema.c++" + "${CAPNPROTO_SOURCE_DIR}/capnp/stream.capnp.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/schema-loader.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/dynamic.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/stringify.c++" diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index b299e1fc00a..d256fe8f160 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -129,8 +129,8 @@ static std::optional convertToDynamicValue(const Co } case capnp::DynamicValue::Type::TEXT: { - auto data = String(column->getDataAt(row_num)); - return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data(), data.size())); + auto data = column->getDataAt(row_num); + return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data, data.size)); } case capnp::DynamicValue::Type::STRUCT: { @@ -220,7 +220,7 @@ void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) auto field_builder = initStructFieldBuilder(columns[i], row_num, struct_builder, field); auto value = convertToDynamicValue(columns[i], column_types[i], row_num, field_builder, format_settings.capn_proto.enum_comparing_mode); if (value) - struct_builder.set(field, std::move(*value)); + struct_builder.set(field, *value); } capnp::writeMessage(*output_stream, message); diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index c24b85109da..23e626d6d96 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -5,11 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') CAPN_PROTO_FILE=$USER_FILES_PATH/data.capnp touch $CAPN_PROTO_FILE -SCHEMADIR=/$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=/$($CLICKHOUSE_CLIENT --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02030 mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR From dd4421d4b1131c246f762646abbd4534aa7a8489 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 5 Oct 2021 14:12:54 +0300 Subject: [PATCH 654/950] Fix build --- contrib/capnproto-cmake/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index 274be8c5eeb..05446355535 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -64,6 +64,7 @@ set (CAPNPC_SRCS "${CAPNPROTO_SOURCE_DIR}/capnp/compiler/lexer.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/compiler/grammar.capnp.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/compiler/parser.c++" + "${CAPNPROTO_SOURCE_DIR}/capnp/compiler/generics.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/compiler/node-translator.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/compiler/compiler.c++" "${CAPNPROTO_SOURCE_DIR}/capnp/schema-parser.c++" From 95790b8a1c25d293b227d2e968a16d5a4d918e68 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Oct 2021 13:51:00 +0300 Subject: [PATCH 655/950] Update CapnProtoUtils.cpp --- src/Formats/CapnProtoUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 2cc20abedd0..59f63243e28 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -43,7 +43,7 @@ capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaIn /// That's not good to determine the type of error by its description, but /// this is the only way to do it here, because kj doesn't specify the type of error. String description = String(e.getDescription().cStr()); - if (description.starts_with("no such directory")) + if (description.starts_with("No such file or directory")) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); if (description.starts_with("Parse error")) From 9ec6930c152af476cbaba2994419c73509b93d9a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 6 Oct 2021 21:12:49 +0300 Subject: [PATCH 656/950] Better exception handling --- src/Formats/CapnProtoUtils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 59f63243e28..1f0e6cf2cac 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -43,10 +43,10 @@ capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaIn /// That's not good to determine the type of error by its description, but /// this is the only way to do it here, because kj doesn't specify the type of error. String description = String(e.getDescription().cStr()); - if (description.starts_with("No such file or directory")) + if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); - if (description.starts_with("Parse error")) + if (description.find("Parse error") != String::npos) throw Exception(ErrorCodes::CANNOT_PARSE_CAPN_PROTO_SCHEMA, "Cannot parse CapnProto schema {}:{}", schema_info.schemaPath(), e.getLine()); throw Exception(ErrorCodes::UNKNOWN_EXCEPTION, "Unknown exception while parsing CapnProro schema: {}, schema dir and file: {}, {}", description, schema_info.schemaDirectory(), schema_info.schemaPath()); From 9b909f3f30f93b44eaf65ee8433733f75abfd99c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 7 Oct 2021 10:58:37 +0300 Subject: [PATCH 657/950] Try to fix test --- tests/queries/0_stateless/02030_capnp_format.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 23e626d6d96..02c4fc96c82 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -5,11 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -USER_FILES_PATH=$($CLICKHOUSE_CLIENT --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') CAPN_PROTO_FILE=$USER_FILES_PATH/data.capnp touch $CAPN_PROTO_FILE -SCHEMADIR=/$($CLICKHOUSE_CLIENT --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") +SCHEMADIR=$(clickhouse-client --query "select * from file('data.capnp', 'CapnProto', 'val1 char') settings format_schema='nonexist:Message'" 2>&1 | grep Exception | grep -oP "file \K.*(?=/nonexist.capnp)") CLIENT_SCHEMADIR=$CURDIR/format_schemas SERVER_SCHEMADIR=test_02030 mkdir -p $SCHEMADIR/$SERVER_SCHEMADIR From f754881e1fd8a42764bfef0b74973abba415808e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 7 Oct 2021 15:28:01 +0300 Subject: [PATCH 658/950] Fix output String data into Text CapnProto type --- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 33 ++++++++++++++----- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index d256fe8f160..2e32c962177 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -88,7 +88,13 @@ static capnp::DynamicValue::Builder initStructFieldBuilder(const ColumnPtr & col return struct_builder.get(field); } -static std::optional convertToDynamicValue(const ColumnPtr & column, const DataTypePtr & data_type, size_t row_num, capnp::DynamicValue::Builder builder, FormatSettings::EnumComparingMode enum_comparing_mode) +static std::optional convertToDynamicValue( + const ColumnPtr & column, + const DataTypePtr & data_type, + size_t row_num, + capnp::DynamicValue::Builder builder, + FormatSettings::EnumComparingMode enum_comparing_mode, + std::vector> & temporary_text_data_storage) { /// Here we don't do any types validation, because we did it in CapnProtoRowOutputFormat constructor. @@ -97,7 +103,7 @@ static std::optional convertToDynamicValue(const Co const auto * lc_column = assert_cast(column.get()); const auto & dict_type = assert_cast(data_type.get())->getDictionaryType(); size_t index = lc_column->getIndexAt(row_num); - return convertToDynamicValue(lc_column->getDictionary().getNestedColumn(), dict_type, index, builder, enum_comparing_mode); + return convertToDynamicValue(lc_column->getDictionary().getNestedColumn(), dict_type, index, builder, enum_comparing_mode, temporary_text_data_storage); } switch (builder.getType()) @@ -129,8 +135,16 @@ static std::optional convertToDynamicValue(const Co } case capnp::DynamicValue::Type::TEXT: { - auto data = column->getDataAt(row_num); - return capnp::DynamicValue::Reader(capnp::Text::Reader(data.data, data.size)); + /// In TEXT type data should be null-terminated, but ClickHouse String data could not be. + /// To make data null-terminated we should copy it to temporary String object, but + /// capnp::Text::Reader works only with pointer to the data and it's size, so we should + /// guarantee that new String object life time is longer than capnp::Text::Reader life time. + /// To do this we store new String object in a temporary storage, passed in this function + /// by reference. We use unique_ptr instead of just String to avoid pointers + /// invalidation on vector reallocation. + temporary_text_data_storage.push_back(std::make_unique(column->getDataAt(row_num))); + auto & data = temporary_text_data_storage.back(); + return capnp::DynamicValue::Reader(capnp::Text::Reader(data->data(), data->size())); } case capnp::DynamicValue::Type::STRUCT: { @@ -153,7 +167,7 @@ static std::optional convertToDynamicValue(const Co struct_builder.clear(value_field); const auto & nested_column = nullable_column->getNestedColumnPtr(); auto value_builder = initStructFieldBuilder(nested_column, row_num, struct_builder, value_field); - auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, value_builder, enum_comparing_mode); + auto value = convertToDynamicValue(nested_column, nullable_type->getNestedType(), row_num, value_builder, enum_comparing_mode, temporary_text_data_storage); if (value) struct_builder.set(value_field, std::move(*value)); } @@ -168,7 +182,7 @@ static std::optional convertToDynamicValue(const Co auto pos = tuple_data_type->getPositionByName(name); auto field_builder = initStructFieldBuilder(nested_columns[pos], row_num, struct_builder, nested_struct_schema.getFieldByName(name)); - auto value = convertToDynamicValue(nested_columns[pos], nested_types[pos], row_num, field_builder, enum_comparing_mode); + auto value = convertToDynamicValue(nested_columns[pos], nested_types[pos], row_num, field_builder, enum_comparing_mode, temporary_text_data_storage); if (value) struct_builder.set(name, std::move(*value)); } @@ -199,7 +213,7 @@ static std::optional convertToDynamicValue(const Co else value_builder = list_builder[i]; - auto value = convertToDynamicValue(nested_column, nested_type, offset + i, value_builder, enum_comparing_mode); + auto value = convertToDynamicValue(nested_column, nested_type, offset + i, value_builder, enum_comparing_mode, temporary_text_data_storage); if (value) list_builder.set(i, std::move(*value)); } @@ -213,12 +227,15 @@ static std::optional convertToDynamicValue(const Co void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) { capnp::MallocMessageBuilder message; + /// Temporary storage for data that will be outputted in fields with CapnProto type TEXT. + /// See comment in convertToDynamicValue() for more details. + std::vector> temporary_text_data_storage; capnp::DynamicStruct::Builder root = message.initRoot(schema); for (size_t i = 0; i != columns.size(); ++i) { auto [struct_builder, field] = getStructBuilderAndFieldByColumnName(root, column_names[i]); auto field_builder = initStructFieldBuilder(columns[i], row_num, struct_builder, field); - auto value = convertToDynamicValue(columns[i], column_types[i], row_num, field_builder, format_settings.capn_proto.enum_comparing_mode); + auto value = convertToDynamicValue(columns[i], column_types[i], row_num, field_builder, format_settings.capn_proto.enum_comparing_mode, temporary_text_data_storage); if (value) struct_builder.set(field, *value); } From 5daed60eaec542047682e279f49ed0c65b8116a2 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 8 Oct 2021 13:23:27 +0300 Subject: [PATCH 659/950] Skip test in case of replicated database --- tests/queries/0_stateless/02030_capnp_format.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02030_capnp_format.sh b/tests/queries/0_stateless/02030_capnp_format.sh index 02c4fc96c82..aa2fe6c1b35 100755 --- a/tests/queries/0_stateless/02030_capnp_format.sh +++ b/tests/queries/0_stateless/02030_capnp_format.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel +# Tags: no-fasttest, no-parallel, no-replicated-database CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 476d7a411f37666adb627206e4a1e11705dea688 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Oct 2021 16:44:28 +0300 Subject: [PATCH 660/950] allow to remove SAMPLE BY expression --- src/Databases/DatabaseOnDisk.cpp | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Parsers/ASTAlterQuery.cpp | 4 ++ src/Parsers/ASTAlterQuery.h | 1 + src/Parsers/ParserAlterQuery.cpp | 5 +++ src/Storages/AlterCommands.cpp | 17 ++++++- src/Storages/AlterCommands.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 11 ++++- .../02097_remove_sample_by.reference | 3 ++ .../0_stateless/02097_remove_sample_by.sql | 44 +++++++++++++++++++ 11 files changed, 88 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02097_remove_sample_by.reference create mode 100644 tests/queries/0_stateless/02097_remove_sample_by.sql diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 97e59f53f64..638aef7186c 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -180,6 +180,8 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo if (metadata.sampling_key.definition_ast) storage_ast.set(storage_ast.sample_by, metadata.sampling_key.definition_ast); + else if (storage_ast.sample_by != nullptr) /// SAMPLE BY was removed + storage_ast.sample_by = nullptr; if (metadata.table_ttl.definition_ast) storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 1c613758ecc..e595bd580b3 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -270,6 +270,7 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_ORDER_BY, database, table); break; } + case ASTAlterCommand::REMOVE_SAMPLE_BY: case ASTAlterCommand::MODIFY_SAMPLE_BY: { required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 53d53bf5ae1..d3153952114 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -157,6 +157,10 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : ""); sample_by->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::REMOVE_SAMPLE_BY) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE SAMPLE BY" << (settings.hilite ? hilite_none : ""); + } else if (type == ASTAlterCommand::ADD_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "") diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 9b40586e09f..3e0d9219549 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -41,6 +41,7 @@ public: RESET_SETTING, MODIFY_QUERY, REMOVE_TTL, + REMOVE_SAMPLE_BY, ADD_INDEX, DROP_INDEX, diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 2eade2079da..1ea64d94fe7 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -104,6 +104,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_ttl("TTL"); ParserKeyword s_remove_ttl("REMOVE TTL"); + ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY"); ParserCompoundIdentifier parser_name; ParserStringLiteral parser_string_literal; @@ -669,6 +670,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MODIFY_SAMPLE_BY; } + else if (s_remove_sample_by.ignore(pos, expected)) + { + command->type = ASTAlterCommand::REMOVE_SAMPLE_BY; + } else if (s_delete.ignore(pos, expected)) { if (s_in_partition.ignore(pos, expected)) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 1d057d1bb10..c5101f162ee 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -205,6 +205,13 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.sample_by = command_ast->sample_by; return command; } + else if (command_ast->type == ASTAlterCommand::REMOVE_SAMPLE_BY) + { + AlterCommand command; + command.ast = command_ast->clone(); + command.type = AlterCommand::REMOVE_SAMPLE_BY; + return command; + } else if (command_ast->type == ASTAlterCommand::ADD_INDEX) { AlterCommand command; @@ -463,6 +470,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { metadata.sampling_key.recalculateWithNewAST(sample_by, metadata.columns, context); } + else if (type == REMOVE_SAMPLE_BY) + { + metadata.sampling_key = {}; + } else if (type == COMMENT_COLUMN) { metadata.columns.modify(column_name, @@ -745,7 +756,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada return false; /// We remove properties on metadata level - if (isRemovingProperty() || type == REMOVE_TTL) + if (isRemovingProperty() || type == REMOVE_TTL || type == REMOVE_SAMPLE_BY) return false; if (type == DROP_COLUMN || type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN) @@ -1208,6 +1219,10 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPt { throw Exception{"Table doesn't have any table TTL expression, cannot remove", ErrorCodes::BAD_ARGUMENTS}; } + else if (command.type == AlterCommand::REMOVE_SAMPLE_BY && !metadata.hasSamplingKey()) + { + throw Exception{"Table doesn't have SAMPLE BY, cannot remove", ErrorCodes::BAD_ARGUMENTS}; + } /// Collect default expressions for MODIFY and ADD comands if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 046238bd5f5..dce6b496741 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -45,7 +45,8 @@ struct AlterCommand RENAME_COLUMN, REMOVE_TTL, MODIFY_DATABASE_SETTING, - COMMENT_TABLE + COMMENT_TABLE, + REMOVE_SAMPLE_BY, }; /// Which property user wants to remove from column diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 790b95a9fa9..3866f760a36 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -249,7 +249,7 @@ MergeTreeData::MergeTreeData( { /// This is for backward compatibility. checkSampleExpression(metadata_, attach || settings->compatibility_allow_sampling_expression_not_in_primary_key, - settings->check_sample_column_is_correct); + settings->check_sample_column_is_correct && !attach); } checkTTLExpressions(metadata_, metadata_); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..5677ae6604f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1016,8 +1016,15 @@ void StorageReplicatedMergeTree::setTableStructure( if (metadata_diff.sampling_expression_changed) { - auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression); - new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext()); + if (!metadata_diff.new_sampling_expression.empty()) + { + auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression); + new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext()); + } + else /// SAMPLE BY was removed + { + new_metadata.sampling_key = {}; + } } if (metadata_diff.skip_indices_changed) diff --git a/tests/queries/0_stateless/02097_remove_sample_by.reference b/tests/queries/0_stateless/02097_remove_sample_by.reference new file mode 100644 index 00000000000..0747bbd5d1f --- /dev/null +++ b/tests/queries/0_stateless/02097_remove_sample_by.reference @@ -0,0 +1,3 @@ +CREATE TABLE default.t_remove_sample_by\n(\n `id` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_remove_sample_by\n(\n `id` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/t_remove_sample_by\', \'1\')\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_remove_sample_by\n(\n `id` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02097_remove_sample_by.sql b/tests/queries/0_stateless/02097_remove_sample_by.sql new file mode 100644 index 00000000000..89fbfe0c4c5 --- /dev/null +++ b/tests/queries/0_stateless/02097_remove_sample_by.sql @@ -0,0 +1,44 @@ +-- Tags: zookeeper + +DROP TABLE IF EXISTS t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id UInt64) ENGINE = MergeTree ORDER BY id SAMPLE BY id; + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; +SHOW CREATE TABLE t_remove_sample_by; + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; -- { serverError 36 } +SELECT * FROM t_remove_sample_by SAMPLE 1 / 10; -- { serverError 141 } + +DROP TABLE t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/t_remove_sample_by', '1') +ORDER BY id SAMPLE BY id; + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; +SHOW CREATE TABLE t_remove_sample_by; + +DROP TABLE t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id UInt64) ENGINE = Memory; +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; -- { serverError 36 } + +DROP TABLE t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id String) +ENGINE = MergeTree ORDER BY id SAMPLE BY id +SETTINGS check_sample_column_is_correct = 0; + +ALTER TABLE t_remove_sample_by RESET SETTING check_sample_column_is_correct; + +DETACH TABLE t_remove_sample_by; +ATTACH TABLE t_remove_sample_by; + +INSERT INTO t_remove_sample_by VALUES (1); +SELECT * FROM t_remove_sample_by SAMPLE 1 / 10; -- { serverError 59 } + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; +SHOW CREATE TABLE t_remove_sample_by; + +DROP TABLE t_remove_sample_by; From 4ebb2455d8ad7e4b3a9bf337e4accd2c2eb45450 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 14 Oct 2021 17:56:34 +0300 Subject: [PATCH 661/950] Delete 01939_network_send_bytes_metrics test --- .../01939_network_send_bytes_metrics.reference | 1 - .../01939_network_send_bytes_metrics.sh | 16 ---------------- 2 files changed, 17 deletions(-) delete mode 100644 tests/queries/0_stateless/01939_network_send_bytes_metrics.reference delete mode 100755 tests/queries/0_stateless/01939_network_send_bytes_metrics.sh diff --git a/tests/queries/0_stateless/01939_network_send_bytes_metrics.reference b/tests/queries/0_stateless/01939_network_send_bytes_metrics.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/01939_network_send_bytes_metrics.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh b/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh deleted file mode 100755 index 840b4f54706..00000000000 --- a/tests/queries/0_stateless/01939_network_send_bytes_metrics.sh +++ /dev/null @@ -1,16 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;" - -${CLICKHOUSE_CLIENT} --query "SELECT number FROM numbers(1000)" > /dev/null - -${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS; - WITH ProfileEvents['NetworkSendBytes'] AS bytes - SELECT bytes >= 8000 AND bytes < 9500 ? 1 : bytes FROM system.query_log - WHERE current_database = currentDatabase() AND query_kind = 'Select' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;" - -${CLICKHOUSE_CLIENT} --query "DROP TABLE t" From 2f6d771d94a764f57b58140de5cc0bcb709f321f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 14 Oct 2021 18:15:30 +0300 Subject: [PATCH 662/950] Remove trailing whitespace --- src/Common/ProgressIndication.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 1f8fc949886..bf3397f50e1 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -126,7 +126,7 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const { return memory + data.second.memory_usage; }); - return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; + return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } From 8d1c51c422ed16ee8a5548f72aba360a73230ffa Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 14 Oct 2021 18:18:04 +0300 Subject: [PATCH 663/950] Update Client.cpp --- programs/client/Client.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 45314a5d460..a5e4bd45c7f 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -89,7 +89,6 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int TOO_DEEP_RECURSION; extern const int NETWORK_ERROR; - extern const int UNRECOGNIZED_ARGUMENTS; extern const int AUTHENTICATION_FAILED; } From 693060552a2ef5a76e109ab05738ca3580a53c3d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?S=C3=A9bastien?= Date: Thu, 14 Oct 2021 17:40:14 +0200 Subject: [PATCH 664/950] add support of window function in antlr grammar --- utils/antlr/ClickHouseLexer.g4 | 8 +++++++ utils/antlr/ClickHouseParser.g4 | 38 +++++++++++++++++++++++---------- 2 files changed, 35 insertions(+), 11 deletions(-) diff --git a/utils/antlr/ClickHouseLexer.g4 b/utils/antlr/ClickHouseLexer.g4 index 8a1debaf412..0c087bff68c 100644 --- a/utils/antlr/ClickHouseLexer.g4 +++ b/utils/antlr/ClickHouseLexer.g4 @@ -35,6 +35,7 @@ CONSTRAINT: C O N S T R A I N T; CREATE: C R E A T E; CROSS: C R O S S; CUBE: C U B E; +CURRENT: C U R R E N T; DATABASE: D A T A B A S E; DATABASES: D A T A B A S E S; DATE: D A T E; @@ -65,6 +66,7 @@ FETCHES: F E T C H E S; FINAL: F I N A L; FIRST: F I R S T; FLUSH: F L U S H; +FOLLOWING: F O L L O W I N G; FOR: F O R; FORMAT: F O R M A T; FREEZE: F R E E Z E; @@ -125,8 +127,10 @@ OR: O R; ORDER: O R D E R; OUTER: O U T E R; OUTFILE: O U T F I L E; +OVER: O V E R; PARTITION: P A R T I T I O N; POPULATE: P O P U L A T E; +PRECEDING: P R E C E D I N G; PREWHERE: P R E W H E R E; PRIMARY: P R I M A R Y; PROJECTION: P R O J E C T I O N; @@ -140,6 +144,8 @@ REPLICA: R E P L I C A; REPLICATED: R E P L I C A T E D; RIGHT: R I G H T; ROLLUP: R O L L U P; +ROW: R O W; +ROWS: R O W S; SAMPLE: S A M P L E; SECOND: S E C O N D; SELECT: S E L E C T; @@ -171,6 +177,7 @@ TRIM: T R I M; TRUNCATE: T R U N C A T E; TTL: T T L; TYPE: T Y P E; +UNBOUNDED: U N B O U N D E D; UNION: U N I O N; UPDATE: U P D A T E; USE: U S E; @@ -183,6 +190,7 @@ WATCH: W A T C H; WEEK: W E E K; WHEN: W H E N; WHERE: W H E R E; +WINDOW: W I N D O W; WITH: W I T H; YEAR: Y E A R | Y Y Y Y; diff --git a/utils/antlr/ClickHouseParser.g4 b/utils/antlr/ClickHouseParser.g4 index eb1908ed073..24db6478aa0 100644 --- a/utils/antlr/ClickHouseParser.g4 +++ b/utils/antlr/ClickHouseParser.g4 @@ -243,6 +243,7 @@ selectStmt: SELECT DISTINCT? topClause? columnExprList fromClause? arrayJoinClause? + windowClause? prewhereClause? whereClause? groupByClause? (WITH (CUBE | ROLLUP))? (WITH TOTALS)? @@ -257,6 +258,7 @@ withClause: WITH columnExprList; topClause: TOP DECIMAL_LITERAL (WITH TIES)?; fromClause: FROM joinExpr; arrayJoinClause: (LEFT | INNER)? ARRAY JOIN columnExprList; +windowClause: WINDOW identifier AS LPAREN windowExpr RPAREN; prewhereClause: PREWHERE columnExpr; whereClause: WHERE columnExpr; groupByClause: GROUP BY ((CUBE | ROLLUP) LPAREN columnExprList RPAREN | columnExprList); @@ -298,6 +300,18 @@ ratioExpr: numberLiteral (SLASH numberLiteral)?; settingExprList: settingExpr (COMMA settingExpr)*; settingExpr: identifier EQ_SINGLE literal; +windowExpr: winPartitionByClause? winOrderByClause? winFrameClause?; +winPartitionByClause: PARTITION BY columnExprList; +winOrderByClause: ORDER BY orderExprList; +winFrameClause: (ROWS | RANGE) winFrameExtend; +winFrameExtend + : winFrameBound # frameStart + | BETWEEN winFrameBound AND winFrameBound # frameBetween + ; +winFrameBound: (CURRENT ROW | UNBOUNDED PRECEDING | UNBOUNDED FOLLOWING | numberLiteral PRECEDING | numberLiteral FOLLOWING); +//rangeClause: RANGE LPAREN (MIN identifier MAX identifier | MAX identifier MIN identifier) RPAREN; + + // SET statement setStmt: SET settingExprList; @@ -364,6 +378,8 @@ columnExpr | SUBSTRING LPAREN columnExpr FROM columnExpr (FOR columnExpr)? RPAREN # ColumnExprSubstring | TIMESTAMP STRING_LITERAL # ColumnExprTimestamp | TRIM LPAREN (BOTH | LEADING | TRAILING) STRING_LITERAL FROM columnExpr RPAREN # ColumnExprTrim + | identifier (LPAREN columnExprList? RPAREN) OVER LPAREN windowExpr RPAREN # ColumnExprWinFunction + | identifier (LPAREN columnExprList? RPAREN) OVER identifier # ColumnExprWinFunctionTarget | identifier (LPAREN columnExprList? RPAREN)? LPAREN DISTINCT? columnArgList? RPAREN # ColumnExprFunction | literal # ColumnExprLiteral @@ -454,17 +470,17 @@ interval: SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR; keyword // except NULL_SQL, INF, NAN_SQL : AFTER | ALIAS | ALL | ALTER | AND | ANTI | ANY | ARRAY | AS | ASCENDING | ASOF | AST | ASYNC | ATTACH | BETWEEN | BOTH | BY | CASE - | CAST | CHECK | CLEAR | CLUSTER | CODEC | COLLATE | COLUMN | COMMENT | CONSTRAINT | CREATE | CROSS | CUBE | DATABASE | DATABASES - | DATE | DEDUPLICATE | DEFAULT | DELAY | DELETE | DESCRIBE | DESC | DESCENDING | DETACH | DICTIONARIES | DICTIONARY | DISK | DISTINCT - | DISTRIBUTED | DROP | ELSE | END | ENGINE | EVENTS | EXISTS | EXPLAIN | EXPRESSION | EXTRACT | FETCHES | FINAL | FIRST | FLUSH | FOR - | FORMAT | FREEZE | FROM | FULL | FUNCTION | GLOBAL | GRANULARITY | GROUP | HAVING | HIERARCHICAL | ID | IF | ILIKE | IN | INDEX - | INJECTIVE | INNER | INSERT | INTERVAL | INTO | IS | IS_OBJECT_ID | JOIN | JSON_FALSE | JSON_TRUE | KEY | KILL | LAST | LAYOUT - | LEADING | LEFT | LIFETIME | LIKE | LIMIT | LIVE | LOCAL | LOGS | MATERIALIZE | MATERIALIZED | MAX | MERGES | MIN | MODIFY | MOVE - | MUTATION | NO | NOT | NULLS | OFFSET | ON | OPTIMIZE | OR | ORDER | OUTER | OUTFILE | PARTITION | POPULATE | PREWHERE | PRIMARY - | RANGE | RELOAD | REMOVE | RENAME | REPLACE | REPLICA | REPLICATED | RIGHT | ROLLUP | SAMPLE | SELECT | SEMI | SENDS | SET | SETTINGS - | SHOW | SOURCE | START | STOP | SUBSTRING | SYNC | SYNTAX | SYSTEM | TABLE | TABLES | TEMPORARY | TEST | THEN | TIES | TIMEOUT - | TIMESTAMP | TOTALS | TRAILING | TRIM | TRUNCATE | TO | TOP | TTL | TYPE | UNION | UPDATE | USE | USING | UUID | VALUES | VIEW - | VOLUME | WATCH | WHEN | WHERE | WITH + | CAST | CHECK | CLEAR | CLUSTER | CODEC | COLLATE | COLUMN | COMMENT | CONSTRAINT | CREATE | CROSS | CUBE | CURRENT | DATABASE + | DATABASES | DATE | DEDUPLICATE | DEFAULT | DELAY | DELETE | DESCRIBE | DESC | DESCENDING | DETACH | DICTIONARIES | DICTIONARY | DISK + | DISTINCT | DISTRIBUTED | DROP | ELSE | END | ENGINE | EVENTS | EXISTS | EXPLAIN | EXPRESSION | EXTRACT | FETCHES | FINAL | FIRST + | FLUSH | FOR | FOLLOWING | FOR | FORMAT | FREEZE | FROM | FULL | FUNCTION | GLOBAL | GRANULARITY | GROUP | HAVING | HIERARCHICAL | ID + | IF | ILIKE | IN | INDEX | INJECTIVE | INNER | INSERT | INTERVAL | INTO | IS | IS_OBJECT_ID | JOIN | JSON_FALSE | JSON_TRUE | KEY + | KILL | LAST | LAYOUT | LEADING | LEFT | LIFETIME | LIKE | LIMIT | LIVE | LOCAL | LOGS | MATERIALIZE | MATERIALIZED | MAX | MERGES + | MIN | MODIFY | MOVE | MUTATION | NO | NOT | NULLS | OFFSET | ON | OPTIMIZE | OR | ORDER | OUTER | OUTFILE | OVER | PARTITION + | POPULATE | PRECEDING | PREWHERE | PRIMARY | RANGE | RELOAD | REMOVE | RENAME | REPLACE | REPLICA | REPLICATED | RIGHT | ROLLUP | ROW + | ROWS | SAMPLE | SELECT | SEMI | SENDS | SET | SETTINGS | SHOW | SOURCE | START | STOP | SUBSTRING | SYNC | SYNTAX | SYSTEM | TABLE + | TABLES | TEMPORARY | TEST | THEN | TIES | TIMEOUT | TIMESTAMP | TOTALS | TRAILING | TRIM | TRUNCATE | TO | TOP | TTL | TYPE + | UNBOUNDED | UNION | UPDATE | USE | USING | UUID | VALUES | VIEW | VOLUME | WATCH | WHEN | WHERE | WINDOW | WITH ; keywordForAlias : DATE | FIRST | ID | KEY From 4800749d32e42912c8c34ab664403d9fea5fa75e Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 14 Oct 2021 23:56:28 +0800 Subject: [PATCH 665/950] make Ctrl-J to commit --- base/base/ReplxxLineReader.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/base/base/ReplxxLineReader.cpp b/base/base/ReplxxLineReader.cpp index 9bf6ec41255..38867faf5d5 100644 --- a/base/base/ReplxxLineReader.cpp +++ b/base/base/ReplxxLineReader.cpp @@ -177,6 +177,10 @@ ReplxxLineReader::ReplxxLineReader( /// 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); }); + + /// bind C-j to ENTER action. + rx.bind_key(Replxx::KEY::control('J'), [this](char32_t code) { return rx.invoke(Replxx::ACTION::COMMIT_LINE, code); }); + /// By default COMPLETE_NEXT/COMPLETE_PREV was binded 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). From 7b1eb7cb54d1dea05be2e7dfb1a2fc3cda7004bc Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 14 Oct 2021 10:16:10 -0600 Subject: [PATCH 666/950] Add 21-10 release blog post --- .../en/2021/clickhouse-v21.10-released.md | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 website/blog/en/2021/clickhouse-v21.10-released.md diff --git a/website/blog/en/2021/clickhouse-v21.10-released.md b/website/blog/en/2021/clickhouse-v21.10-released.md new file mode 100644 index 00000000000..720ff0cc6d1 --- /dev/null +++ b/website/blog/en/2021/clickhouse-v21.10-released.md @@ -0,0 +1,27 @@ +--- +title: 'ClickHouse v21.10 Released' +image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-10/featured.jpg' +date: '2021-10-14' +author: '[Alexey Milovidov](https://github.com/alexey-milovidov)' +tags: ['company', 'community'] +--- + +We're excited to share with you our first release since [announcing ClickHouse, Inc](https://clickhouse.com/blog/en/2021/clickhouse-inc/). The 21.10 release includes new contributions from multiple contributors including many in our community, and we are grateful for your ongoing ideas, development, and support. Our Engineering team continues to be laser-focused on providing our community and users with the fastest and most scalable OLAP DBMS available while implementing many new features. In the 21.10 release, we have a wonderful 79 contributors with 1255 commits across 211 pull requests - what an amazing community and we cherish your contributions. + +Let's highlight some of these new exciting new capabilities in 21.10: + +* User-defined functions (UDFs) can now be [created as lambda expressions](https://clickhouse.com/docs/en/sql-reference/functions/#higher-order-functions). For example, `CREATE FUNCTION plus_one as (a) -> a + 1` +* Two new table engines: Executable and ExecutablePool which allow you to stream the results of a query to a custom shell script +* Instead of logging every query (which can be a lot of logs!), you can now log a random sample of your queries. The number of queries logged is determined by defining a specified probability between 0.0 (no queries logged) and 1.0 (all queries logged) using the new `log_queries_probability` setting. +* Positional arguments are now available in your GROUP BY, ORDER BY and LIMIT BY clauses. For example, `SELECT foo, bar, baz FROM my_table ORDER BY 2,3` orders the results by whatever the bar and baz columns (no need to specify column names twice!) + +We're always listening for new ideas, and we're happy to welcome new contributors to the ClickHouse project. Whether for submitting code or improving our documentation and examples, please get involved by sending us a pull request or submitting an issue. Our beginner developers contribution guide will help you get started [[https://clickhouse.com/docs/en/development/developer-instruction/](https://clickhouse.com/docs/en/development/developer-instruction/)] + + +## ClickHouse Release Notes + +Release 21.10 + +Release Date: 2021-10-19 + +Release Notes: [21.10](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) From 4dd1a4fe5c8e087cde635c28a19667937c1dc095 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 14 Oct 2021 10:41:41 -0600 Subject: [PATCH 667/950] Update release date --- website/blog/en/2021/clickhouse-v21.10-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.10-released.md b/website/blog/en/2021/clickhouse-v21.10-released.md index 720ff0cc6d1..e5d870147f4 100644 --- a/website/blog/en/2021/clickhouse-v21.10-released.md +++ b/website/blog/en/2021/clickhouse-v21.10-released.md @@ -22,6 +22,6 @@ We're always listening for new ideas, and we're happy to welcome new contributor Release 21.10 -Release Date: 2021-10-19 +Release Date: 2021-10-21 Release Notes: [21.10](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) From 56ab0e31e63277d6b8cf288c35d0c00fe58ba108 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:45:45 +0300 Subject: [PATCH 668/950] Add description for test_MemoryTracking --- tests/integration/test_MemoryTracking/test.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/tests/integration/test_MemoryTracking/test.py b/tests/integration/test_MemoryTracking/test.py index 59b30b8f08b..51a7214a442 100644 --- a/tests/integration/test_MemoryTracking/test.py +++ b/tests/integration/test_MemoryTracking/test.py @@ -2,6 +2,17 @@ # pylint: disable=redefined-outer-name # pylint: disable=line-too-long +# This test verifies that memory tracking does not have significant drift, +# in other words, every allocation should be taken into account at the global +# memory tracker. +# +# So we are running some queries with GROUP BY to make some allocations, +# and after we are checking MemoryTracking metric from system.metrics, +# and check that it does not changes too frequently. +# +# Also note, that syncing MemoryTracking with RSS had been disabled in +# asynchronous_metrics_update_period_s.xml. + import logging import pytest from helpers.cluster import ClickHouseCluster From d030e08d05c3257ef9d168052d723be0105e22ac Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:45:45 +0300 Subject: [PATCH 669/950] Make test_MemoryTracking::test_http not flaky By comparing only megabytes in the memory changes, instead of bytes as before, since it may be tricky at least due to max_untracked_memory and how thread pool handle it. It should be safe, since originally it was written in #16121 which fixes issue #15932, which has ~4MB consumption of memory per request. --- tests/integration/test_MemoryTracking/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_MemoryTracking/test.py b/tests/integration/test_MemoryTracking/test.py index 51a7214a442..2ec5b2457af 100644 --- a/tests/integration/test_MemoryTracking/test.py +++ b/tests/integration/test_MemoryTracking/test.py @@ -57,6 +57,8 @@ def get_MemoryTracking(): return int(http_query("SELECT value FROM system.metrics WHERE metric = 'MemoryTracking'")) def check_memory(memory): + # bytes -> megabytes + memory = [*map(lambda x: int(int(x)/1024/1024), memory)] # 3 changes to MemoryTracking is minimum, since: # - this is not that high to not detect inacuracy # - memory can go like X/X+N due to some background allocations From e9cab3aaad1adeb19b0504181dc359d2ce15ac69 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 21:20:19 +0300 Subject: [PATCH 670/950] Smaller smoothing window in throttler. --- src/Common/Throttler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index fd434922ac2..f02001e338a 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -23,7 +23,7 @@ static constexpr auto NS = 1000000000UL; /// Tracking window. Actually the size is not really important. We just want to avoid /// throttles when there are no actions for a long period time. -static const double window_ns = 7UL * NS; +static const double window_ns = 1UL * NS; void Throttler::add(size_t amount) { From aa56fd15925f53fc0b9ca5b6b0d0604a492b6172 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 21:29:02 +0300 Subject: [PATCH 671/950] Fix style. --- src/TableFunctions/ITableFunctionFileLike.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 49461fe8f46..699ad698bd8 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -16,8 +16,6 @@ #include - - namespace DB { From 89c1a04ef4eb2819631266f6051a1dfe0c818ecb Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 21:35:56 +0300 Subject: [PATCH 672/950] Fix comments --- src/Formats/CapnProtoUtils.cpp | 118 ++++++++++-------- src/Formats/registerFormats.cpp | 2 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 27 ++-- .../Formats/Impl/CapnProtoRowOutputFormat.cpp | 4 +- 4 files changed, 85 insertions(+), 66 deletions(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 1f0e6cf2cac..4b9993d5a74 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -42,7 +42,7 @@ capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaIn { /// That's not good to determine the type of error by its description, but /// this is the only way to do it here, because kj doesn't specify the type of error. - String description = String(e.getDescription().cStr()); + auto description = std::string_view(e.getDescription().cStr()); if (description.find("No such file or directory") != String::npos || description.find("no such directory") != String::npos) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot open CapnProto schema, file {} doesn't exists", schema_info.absoluteSchemaPath()); @@ -82,7 +82,6 @@ static const std::map capnp_simple_type_name {capnp::schema::Type::Which::FLOAT64, "Float64"}, {capnp::schema::Type::Which::TEXT, "Text"}, {capnp::schema::Type::Which::DATA, "Data"}, - {capnp::schema::Type::Which::ENUM, "Enum"}, {capnp::schema::Type::Which::INTERFACE, "Interface"}, {capnp::schema::Type::Which::ANY_POINTER, "AnyPointer"}, }; @@ -100,40 +99,56 @@ static bool checkIfStructIsNamedUnion(const capnp::StructSchema & struct_schema) /// Get full name of type for better exception messages. static String getCapnProtoFullTypeName(const capnp::Type & type) { - if (type.isStruct()) + switch (type.which()) { - auto struct_schema = type.asStruct(); + case capnp::schema::Type::Which::STRUCT: + { + auto struct_schema = type.asStruct(); - auto non_union_fields = struct_schema.getNonUnionFields(); - std::vector non_union_field_names; - for (auto nested_field : non_union_fields) - non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + auto non_union_fields = struct_schema.getNonUnionFields(); + std::vector non_union_field_names; + for (auto nested_field : non_union_fields) + non_union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - auto union_fields = struct_schema.getUnionFields(); - std::vector union_field_names; - for (auto nested_field : union_fields) - union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); + auto union_fields = struct_schema.getUnionFields(); + std::vector union_field_names; + for (auto nested_field : union_fields) + union_field_names.push_back(String(nested_field.getProto().getName()) + " " + getCapnProtoFullTypeName(nested_field.getType())); - String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; - /// Check if the struct is a named union. - if (non_union_field_names.empty()) - return union_name; + String union_name = "Union(" + boost::algorithm::join(union_field_names, ", ") + ")"; + /// Check if the struct is a named union. + if (non_union_field_names.empty()) + return union_name; - String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); - /// Check if the struct contains unnamed union. - if (!union_field_names.empty()) - type_name += "," + union_name; - type_name += ")"; - return type_name; + String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); + /// Check if the struct contains unnamed union. + if (!union_field_names.empty()) + type_name += "," + union_name; + type_name += ")"; + return type_name; + } + case capnp::schema::Type::Which::LIST: + return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; + case capnp::schema::Type::Which::ENUM: + { + auto enum_schema = type.asEnum(); + String enum_name = "Enum("; + auto enumerants = enum_schema.getEnumerants(); + for (size_t i = 0; i != enumerants.size(); ++i) + { + enum_name += String(enumerants[i].getProto().getName()) + " = " + std::to_string(enumerants[i].getOrdinal()); + if (i + 1 != enumerants.size()) + enum_name += ", "; + } + enum_name += ")"; + return enum_name; + } + default: + auto it = capnp_simple_type_names.find(type.which()); + if (it == capnp_simple_type_names.end()) + throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); + return it->second; } - - if (type.isList()) - return "List(" + getCapnProtoFullTypeName(type.asList().getElementType()) + ")"; - - if (!capnp_simple_type_names.contains(type.which())) - throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD, "Unknown CapnProto type"); - - return capnp_simple_type_names.at(type.which()); } template @@ -147,39 +162,38 @@ static bool checkEnums(const capnp::Type & capnp_type, const DataTypePtr column_ const auto * enum_type = assert_cast *>(column_type.get()); const auto & enum_values = dynamic_cast &>(*enum_type); - auto names = enum_values.getSetOfAllNames(to_lower); - auto values = enum_values.getSetOfAllValues(); - - std::unordered_set capn_enum_names; - std::unordered_set capn_enum_values; - auto enumerants = enum_schema.getEnumerants(); - /// In CapnProto Enum fields are numbered sequentially starting from zero. - if (mode == FormatSettings::EnumComparingMode::BY_VALUES && enumerants.size() > max_value) + if (mode == FormatSettings::EnumComparingMode::BY_VALUES) { - error_message += "Enum from CapnProto schema contains values that is out of range for Clickhouse Enum"; - return false; + /// In CapnProto Enum fields are numbered sequentially starting from zero. + if (enumerants.size() > max_value) + { + error_message += "Enum from CapnProto schema contains values that is out of range for Clickhouse Enum"; + return false; + } + + auto values = enum_values.getSetOfAllValues(); + std::unordered_set capn_enum_values; + for (auto enumerant : enumerants) + capn_enum_values.insert(Type(enumerant.getOrdinal())); + auto result = values == capn_enum_values; + if (!result) + error_message += "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"; + return result; } + auto names = enum_values.getSetOfAllNames(to_lower); + std::unordered_set capn_enum_names; + for (auto enumerant : enumerants) { String name = enumerant.getProto().getName(); capn_enum_names.insert(to_lower ? boost::algorithm::to_lower_copy(name) : name); - auto value = enumerant.getOrdinal(); - capn_enum_values.insert(Type(value)); } - if (mode == FormatSettings::EnumComparingMode::BY_NAMES || mode == FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE) - { - auto result = names == capn_enum_names; - if (!result) - error_message += "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"; - return result; - } - - auto result = values == capn_enum_values; + auto result = names == capn_enum_names; if (!result) - error_message += "The set of values in Enum from CapnProto schema is different from the set of values in ClickHouse Enum"; + error_message += "The set of names in Enum from CapnProto schema is different from the set of names in ClickHouse Enum"; return result; } diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index f6b4bb7e2e1..acaf6f28492 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -140,7 +140,7 @@ void registerFormats() registerOutputFormatMySQLWire(factory); registerOutputFormatMarkdown(factory); registerOutputFormatPostgreSQLWire(factory); - registerOutputFormatProcessorsCapnProto(factory); + registerOutputFormatCapnProto(factory); registerInputFormatRegexp(factory); registerInputFormatJSONAsString(factory); diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp index 8492fc9b623..4d000bb1f35 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.cpp @@ -148,20 +148,25 @@ static void insertEnum(IColumn & column, const DataTypePtr & column_type, const auto enumerant = *kj::_::readMaybe(enum_value.getEnumerant()); auto enum_type = assert_cast *>(column_type.get()); DataTypePtr nested_type = std::make_shared>(); - if (enum_comparing_mode == FormatSettings::EnumComparingMode::BY_VALUES) - insertSignedInteger(column, nested_type, Int64(enumerant.getOrdinal())); - else if (enum_comparing_mode == FormatSettings::EnumComparingMode::BY_NAMES) - insertSignedInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName())))); - else + switch (enum_comparing_mode) { - /// Find the same enum name case insensitive. - String enum_name = enumerant.getProto().getName(); - for (auto & name : enum_type->getAllRegisteredNames()) + case FormatSettings::EnumComparingMode::BY_VALUES: + insertSignedInteger(column, nested_type, Int64(enumerant.getOrdinal())); + return; + case FormatSettings::EnumComparingMode::BY_NAMES: + insertSignedInteger(column, nested_type, Int64(enum_type->getValue(String(enumerant.getProto().getName())))); + return; + case FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE: { - if (compareEnumNames(name, enum_name, enum_comparing_mode)) + /// Find the same enum name case insensitive. + String enum_name = enumerant.getProto().getName(); + for (auto & name : enum_type->getAllRegisteredNames()) { - insertSignedInteger(column, nested_type, Int64(enum_type->getValue(name))); - break; + if (compareEnumNames(name, enum_name, enum_comparing_mode)) + { + insertSignedInteger(column, nested_type, Int64(enum_type->getValue(name))); + break; + } } } } diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index 2e32c962177..b5e2b83c23b 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -243,9 +243,9 @@ void CapnProtoRowOutputFormat::write(const Columns & columns, size_t row_num) capnp::writeMessage(*output_stream, message); } -void registerOutputFormatProcessorsCapnProto(FormatFactory & factory) +void registerOutputFormatCapnProto(FormatFactory & factory) { - factory.registerOutputFormatProcessor("CapnProto", []( + factory.registerOutputFormat("CapnProto", []( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, From 8729201208c374a27df726233e5c17515f2ffb95 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 21:36:57 +0300 Subject: [PATCH 673/950] Remove redundant move --- src/Formats/CapnProtoUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 4b9993d5a74..b9a28bd3fb3 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -422,7 +422,7 @@ void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Blo getCapnProtoFullTypeName(field.getType())); if (!additional_error_message.empty()) e.addMessage(additional_error_message); - throw std::move(e); + throw e; } } } From 2da8180613a106e26d091497cda1fc52d8cb905a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 21:39:09 +0300 Subject: [PATCH 674/950] Add space after comma --- src/Formats/CapnProtoUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index b9a28bd3fb3..1dc37ff51ec 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -123,7 +123,7 @@ static String getCapnProtoFullTypeName(const capnp::Type & type) String type_name = "Struct(" + boost::algorithm::join(non_union_field_names, ", "); /// Check if the struct contains unnamed union. if (!union_field_names.empty()) - type_name += "," + union_name; + type_name += ", " + union_name; type_name += ")"; return type_name; } From 700504e7d6c24f268ab97ac8055db1dcf6228da6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Oct 2021 22:09:44 +0300 Subject: [PATCH 675/950] Update Internals.cpp --- programs/copier/Internals.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 84283777c8f..6fc69361c90 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -74,7 +74,6 @@ Block getBlockWithAllStreamData(QueryPipeline pipeline) return block; } - bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) { const auto & storage = storage_ast->as(); From 83879415b4d4ae7927952861773ec09b0c95f794 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 22:12:06 +0300 Subject: [PATCH 676/950] Fixed tests --- src/Core/QualifiedTableName.h | 14 ++++++ src/Interpreters/AddDefaultDatabaseVisitor.h | 18 +++----- .../ExternalDictionariesLoader.cpp | 21 ++++++--- src/Interpreters/ExternalDictionariesLoader.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 1 - ...97_default_dict_get_add_database.reference | 1 + .../02097_default_dict_get_add_database.sql | 43 +++++++++++++++++++ 7 files changed, 78 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02097_default_dict_get_add_database.reference create mode 100755 tests/queries/0_stateless/02097_default_dict_get_add_database.sql diff --git a/src/Core/QualifiedTableName.h b/src/Core/QualifiedTableName.h index c1cb9b27d15..cd72dc2809d 100644 --- a/src/Core/QualifiedTableName.h +++ b/src/Core/QualifiedTableName.h @@ -40,6 +40,20 @@ struct QualifiedTableName return hash_state.get64(); } + std::vector getParts() const { + if (database.empty()) + return {table}; + else + return {database, table}; + } + + std::string getFullName() const { + if (database.empty()) + return table; + else + return database + '.' + table; + } + /// NOTE: It's different from compound identifier parsing and does not support escaping and dots in name. /// Usually it's better to use ParserIdentifier instead, /// but we parse DDL dictionary name (and similar things) this way for historical reasons. diff --git a/src/Interpreters/AddDefaultDatabaseVisitor.h b/src/Interpreters/AddDefaultDatabaseVisitor.h index f4b09d5a761..98d33db3021 100644 --- a/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -143,21 +143,12 @@ private: { if (auto * identifier = child->children[i]->as()) { + /// Identifier already qualified if (identifier->compound()) continue; - auto storage_id = context->getExternalDictionariesLoader().getStorageID(identifier->name(), context); - - if (!storage_id.database_name.empty()) - { - std::vector name_parts = {storage_id.database_name, storage_id.table_name}; - child->children[i] = std::make_shared(std::move(name_parts)); - } - else - { - std::vector name_parts = {storage_id.table_name}; - child->children[i] = std::make_shared(std::move(name_parts)); - } + auto qualified_dictionary_name = context->getExternalDictionariesLoader().qualifyDictionaryNameWithDatabase(identifier->name(), context); + child->children[i] = std::make_shared(qualified_dictionary_name.getParts()); } else if (auto * literal = child->children[i]->as()) { @@ -167,7 +158,8 @@ private: continue; auto dictionary_name = literal_value.get(); - literal_value = context->getExternalDictionariesLoader().getStorageID(dictionary_name, context).getFullTableName(); + auto qualified_dictionary_name = context->getExternalDictionariesLoader().qualifyDictionaryNameWithDatabase(dictionary_name, context); + literal_value = qualified_dictionary_name.getFullName(); } } else if (is_operator_in && i == 1) diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index c49b037ef96..c09ab8b78e5 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -87,19 +87,26 @@ DictionaryStructure ExternalDictionariesLoader::getDictionaryStructure(const std return ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); } -StorageID ExternalDictionariesLoader::getStorageID(const std::string & dictionary_name, ContextPtr context) const +QualifiedTableName ExternalDictionariesLoader::qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr query_context) const { - if (has(dictionary_name)) - return StorageID("", dictionary_name); - auto qualified_name = QualifiedTableName::tryParseFromString(dictionary_name); if (!qualified_name) - return StorageID("", dictionary_name); + { + QualifiedTableName qualified_dictionary_name; + qualified_dictionary_name.table = dictionary_name; + return qualified_dictionary_name; + } + + if (qualified_name->database.empty() && has(dictionary_name)) + { + /// This is xml dictionary + return *qualified_name; + } if (qualified_name->database.empty()) - return StorageID(context->getCurrentDatabase(), dictionary_name); + qualified_name->database = query_context->getCurrentDatabase(); - return StorageID("", dictionary_name); + return *qualified_name; } std::string ExternalDictionariesLoader::resolveDictionaryName(const std::string & dictionary_name, const std::string & current_database_name) const diff --git a/src/Interpreters/ExternalDictionariesLoader.h b/src/Interpreters/ExternalDictionariesLoader.h index fea2c363f9a..421154a6d4f 100644 --- a/src/Interpreters/ExternalDictionariesLoader.h +++ b/src/Interpreters/ExternalDictionariesLoader.h @@ -27,7 +27,7 @@ public: void reloadDictionary(const std::string & dictionary_name, ContextPtr context) const; - StorageID getStorageID(const std::string & dictionary_name, ContextPtr context) const; + QualifiedTableName qualifyDictionaryNameWithDatabase(const std::string & dictionary_name, ContextPtr context) const; DictionaryStructure getDictionaryStructure(const std::string & dictionary_name, ContextPtr context) const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e1b2f9097d6..6d38c55bd62 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -924,7 +924,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) visitor.visit(*create.select); } - if (create.columns_list) { AddDefaultDatabaseVisitor visitor(getContext(), current_database); diff --git a/tests/queries/0_stateless/02097_default_dict_get_add_database.reference b/tests/queries/0_stateless/02097_default_dict_get_add_database.reference new file mode 100644 index 00000000000..9b0ac07a68a --- /dev/null +++ b/tests/queries/0_stateless/02097_default_dict_get_add_database.reference @@ -0,0 +1 @@ +CREATE TABLE `02097_db`.test_table_default (`data_1` UInt64 DEFAULT dictGetUInt64(\'02097_db.test_dictionary\', \'data_column_1\', toUInt64(0)), `data_2` UInt8 DEFAULT dictGet(`02097_db`.test_dictionary, \'data_column_2\', toUInt64(0))) ENGINE = TinyLog diff --git a/tests/queries/0_stateless/02097_default_dict_get_add_database.sql b/tests/queries/0_stateless/02097_default_dict_get_add_database.sql new file mode 100755 index 00000000000..d3e4bb6752d --- /dev/null +++ b/tests/queries/0_stateless/02097_default_dict_get_add_database.sql @@ -0,0 +1,43 @@ +-- Tags: no-parallel + +DROP DATABASE IF EXISTS 02097_db; +CREATE DATABASE 02097_db; + +USE 02097_db; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + key_column UInt64, + data_column_1 UInt64, + data_column_2 UInt8 +) +ENGINE = MergeTree +ORDER BY key_column; + +DROP DICTIONARY IF EXISTS test_dictionary; +CREATE DICTIONARY test_dictionary +( + key_column UInt64 DEFAULT 0, + data_column_1 UInt64 DEFAULT 1, + data_column_2 UInt8 DEFAULT 1 +) +PRIMARY KEY key_column +LAYOUT(DIRECT()) +SOURCE(CLICKHOUSE(TABLE 'test_table')); + +DROP TABLE IF EXISTS test_table_default; +CREATE TABLE test_table_default +( + data_1 DEFAULT dictGetUInt64('test_dictionary', 'data_column_1', toUInt64(0)), + data_2 DEFAULT dictGet(test_dictionary, 'data_column_2', toUInt64(0)) +) +ENGINE=TinyLog; + +SELECT create_table_query FROM system.tables WHERE name = 'test_table_default' AND database = '02097_db'; + +DROP DICTIONARY test_dictionary; +DROP TABLE test_table; +DROP TABLE test_table_default; + +DROP DATABASE 02097_db; From 913a8f3640895e833c4905763eb7a2a956a0f72a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 22:15:21 +0300 Subject: [PATCH 677/950] Fixed style check --- src/Core/QualifiedTableName.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Core/QualifiedTableName.h b/src/Core/QualifiedTableName.h index cd72dc2809d..4642465f461 100644 --- a/src/Core/QualifiedTableName.h +++ b/src/Core/QualifiedTableName.h @@ -40,14 +40,16 @@ struct QualifiedTableName return hash_state.get64(); } - std::vector getParts() const { + std::vector getParts() const + { if (database.empty()) return {table}; else return {database, table}; } - std::string getFullName() const { + std::string getFullName() const + { if (database.empty()) return table; else From 74e3978110f8d4eb8b06919e6bad3f07017e11f2 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Thu, 14 Oct 2021 22:55:17 +0300 Subject: [PATCH 678/950] commit assert to fix build --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index a139bb3cb95..ea9add0d76b 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit a139bb3cb9598c7d92dc69aa6962e3ea4fd18982 +Subproject commit ea9add0d76b0d2ff8616c5e9035389cd159996b7 From 5568461acf2c4d6feed6244a711d0e7428c5f773 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 23:10:53 +0300 Subject: [PATCH 679/950] FunctionsJSON move to cpp file --- src/Functions/FunctionsJSON.cpp | 1455 ++++++++++++++++++++++++++++++- src/Functions/FunctionsJSON.h | 1390 ----------------------------- 2 files changed, 1421 insertions(+), 1424 deletions(-) delete mode 100644 src/Functions/FunctionsJSON.h diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 7fec45d1f72..12b4ca76ec2 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1,60 +1,1447 @@ -#include +#include + +#include + +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include +#include +#include +#include +#include +#include + +#include + + +#if !defined(ARCADIA_BUILD) +# include "config_functions.h" +#endif namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -std::vector FunctionJSONHelpers::prepareMoves(const char * function_name, const ColumnsWithTypeAndName & columns, size_t first_index_argument, size_t num_index_arguments) -{ - std::vector moves; - moves.reserve(num_index_arguments); - for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) - { - const auto & column = columns[i]; - if (!isString(column.type) && !isInteger(column.type)) - throw Exception{"The argument " + std::to_string(i + 1) + " of function " + String(function_name) - + " should be a string specifying key or an integer specifying index, illegal type: " + column.type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; +/// Functions to parse JSONs and extract values from it. +/// The first argument of all these functions gets a JSON, +/// after that there are any number of arguments specifying path to a desired part from the JSON's root. +/// For example, +/// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 - if (column.column && isColumnConst(*column.column)) +class FunctionJSONHelpers +{ +public: + template typename Impl, class JSONParser> + class Executor + { + public: + static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) { - const auto & column_const = assert_cast(*column.column); - if (isString(column.type)) - moves.emplace_back(MoveType::ConstKey, column_const.getValue()); + MutableColumnPtr to{result_type->createColumn()}; + to->reserve(input_rows_count); + + if (arguments.empty()) + throw Exception{"Function " + String(Name::name) + " requires at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + const auto & first_column = arguments[0]; + if (!isString(first_column.type)) + throw Exception{"The first argument of function " + String(Name::name) + " should be a string containing JSON, illegal type: " + first_column.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + const ColumnPtr & arg_json = first_column.column; + const auto * col_json_const = typeid_cast(arg_json.get()); + const auto * col_json_string + = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); + + if (!col_json_string) + throw Exception{"Illegal column " + arg_json->getName(), ErrorCodes::ILLEGAL_COLUMN}; + + const ColumnString::Chars & chars = col_json_string->getChars(); + const ColumnString::Offsets & offsets = col_json_string->getOffsets(); + + size_t num_index_arguments = Impl::getNumberOfIndexArguments(arguments); + std::vector moves = prepareMoves(Name::name, arguments, 1, num_index_arguments); + + /// Preallocate memory in parser if necessary. + JSONParser parser; + if constexpr (has_member_function_reserve::value) + { + size_t max_size = calculateMaxSize(offsets); + if (max_size) + parser.reserve(max_size); + } + + Impl impl; + + /// prepare() does Impl-specific preparation before handling each row. + if constexpr (has_member_function_prepare::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value) + impl.prepare(Name::name, arguments, result_type); + + using Element = typename JSONParser::Element; + + Element document; + bool document_ok = false; + if (col_json_const) + { + std::string_view json{reinterpret_cast(&chars[0]), offsets[0] - 1}; + document_ok = parser.parse(json, document); + } + + for (const auto i : collections::range(0, input_rows_count)) + { + if (!col_json_const) + { + std::string_view json{reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1}; + document_ok = parser.parse(json, document); + } + + bool added_to_column = false; + if (document_ok) + { + /// Perform moves. + Element element; + std::string_view last_key; + bool moves_ok = performMoves(arguments, i, document, moves, element, last_key); + + if (moves_ok) + added_to_column = impl.insertResultToColumn(*to, element, last_key); + } + + /// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions. + if (!added_to_column) + to->insertDefault(); + } + return to; + } + }; + +private: + BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) + BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) + + template + struct has_index_operator : std::false_type {}; + + template + struct has_index_operator()[0])>> : std::true_type {}; + + /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. + /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) + /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. + /// Keys and indices can be nonconst, in this case they are calculated for each row. + enum class MoveType + { + Key, + Index, + ConstKey, + ConstIndex, + }; + + struct Move + { + Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} + Move(MoveType type_, const String & key_) : type(type_), key(key_) {} + MoveType type; + size_t index = 0; + String key; + }; + + static std::vector prepareMoves( + const char * function_name, + const ColumnsWithTypeAndName & columns, + size_t first_index_argument, + size_t num_index_arguments) + { + std::vector moves; + moves.reserve(num_index_arguments); + for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) + { + const auto & column = columns[i]; + if (!isString(column.type) && !isInteger(column.type)) + throw Exception{"The argument " + std::to_string(i + 1) + " of function " + String(function_name) + + " should be a string specifying key or an integer specifying index, illegal type: " + column.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (column.column && isColumnConst(*column.column)) + { + const auto & column_const = assert_cast(*column.column); + if (isString(column.type)) + moves.emplace_back(MoveType::ConstKey, column_const.getValue()); + else + moves.emplace_back(MoveType::ConstIndex, column_const.getInt(0)); + } else - moves.emplace_back(MoveType::ConstIndex, column_const.getInt(0)); + { + if (isString(column.type)) + moves.emplace_back(MoveType::Key, ""); + else + moves.emplace_back(MoveType::Index, 0); + } + } + return moves; + } + + + /// Performs moves of types MoveType::Index and MoveType::ConstIndex. + template + static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row, + const typename JSONParser::Element & document, const std::vector & moves, + typename JSONParser::Element & element, std::string_view & last_key) + { + typename JSONParser::Element res_element = document; + std::string_view key; + + for (size_t j = 0; j != moves.size(); ++j) + { + switch (moves[j].type) + { + case MoveType::ConstIndex: + { + if (!moveToElementByIndex(res_element, moves[j].index, key)) + return false; + break; + } + case MoveType::ConstKey: + { + key = moves[j].key; + if (!moveToElementByKey(res_element, key)) + return false; + break; + } + case MoveType::Index: + { + Int64 index = (*arguments[j + 1].column)[row].get(); + if (!moveToElementByIndex(res_element, index, key)) + return false; + break; + } + case MoveType::Key: + { + key = std::string_view{(*arguments[j + 1].column).getDataAt(row)}; + if (!moveToElementByKey(res_element, key)) + return false; + break; + } + } + } + + element = res_element; + last_key = key; + return true; + } + + template + static bool moveToElementByIndex(typename JSONParser::Element & element, int index, std::string_view & out_key) + { + if (element.isArray()) + { + auto array = element.getArray(); + if (index >= 0) + --index; + else + index += array.size(); + + if (static_cast(index) >= array.size()) + return false; + element = array[index]; + out_key = {}; + return true; + } + + if constexpr (has_index_operator::value) + { + if (element.isObject()) + { + auto object = element.getObject(); + if (index >= 0) + --index; + else + index += object.size(); + + if (static_cast(index) >= object.size()) + return false; + std::tie(out_key, element) = object[index]; + return true; + } + } + + return {}; + } + + /// Performs moves of types MoveType::Key and MoveType::ConstKey. + template + static bool moveToElementByKey(typename JSONParser::Element & element, const std::string_view & key) + { + if (!element.isObject()) + return false; + auto object = element.getObject(); + return object.find(key, element); + } + + static size_t calculateMaxSize(const ColumnString::Offsets & offsets) + { + size_t max_size = 0; + for (const auto i : collections::range(0, offsets.size())) + { + size_t size = offsets[i] - offsets[i - 1]; + if (max_size < size) + max_size = size; + } + if (max_size) + --max_size; + return max_size; + } + +}; + + +template typename Impl> +class ExecutableFunctionJSON : public IExecutableFunction, WithContext +{ + +public: + explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_) + : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_) + { + } + + String getName() const override { return Name::name; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + if (null_presence.has_null_constant) + return result_type->createColumnConstWithDefaultValue(input_rows_count); + + ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; + ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); + if (null_presence.has_nullable) + return wrapInNullable(temporary_result, arguments, result_type, input_rows_count); + return temporary_result; + } + +private: + + ColumnPtr + chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const + { +#if USE_SIMDJSON + if (allow_simdjson) + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +#endif + +#if USE_RAPIDJSON + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +#else + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +#endif + } + + NullPresence null_presence; + bool allow_simdjson; + DataTypePtr json_return_type; +}; + + +template typename Impl> +class FunctionBaseFunctionJSON : public IFunctionBase +{ +public: + explicit FunctionBaseFunctionJSON( + const NullPresence & null_presence_, + bool allow_simdjson_, + DataTypes argument_types_, + DataTypePtr return_type_, + DataTypePtr json_return_type_) + : null_presence(null_presence_) + , allow_simdjson(allow_simdjson_) + , argument_types(std::move(argument_types_)) + , return_type(std::move(return_type_)) + , json_return_type(std::move(json_return_type_)) + { + } + + String getName() const override { return Name::name; } + + const DataTypes & getArgumentTypes() const override + { + return argument_types; + } + + const DataTypePtr & getResultType() const override + { + return return_type; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique>(null_presence, allow_simdjson, json_return_type); + } + +private: + NullPresence null_presence; + bool allow_simdjson; + DataTypes argument_types; + DataTypePtr return_type; + DataTypePtr json_return_type; +}; + + +/// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. +/// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. +template typename Impl> +class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext +{ +public: + static constexpr auto name = Name::name; + + String getName() const override { return name; } + + static FunctionOverloadResolverPtr create(ContextPtr context_) + { + return std::make_unique(context_); + } + + explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForNulls() const override { return false; } + + FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override + { + DataTypePtr json_return_type = Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments)); + NullPresence null_presence = getNullPresense(arguments); + DataTypePtr return_type; + if (null_presence.has_null_constant) + return_type = makeNullable(std::make_shared()); + else if (null_presence.has_nullable) + return_type = makeNullable(json_return_type); + else + return_type = json_return_type; + + /// Top-level LowCardinality columns are processed outside JSON parser. + json_return_type = removeLowCardinality(json_return_type); + + DataTypes argument_types; + argument_types.reserve(arguments.size()); + for (const auto & argument : arguments) + argument_types.emplace_back(argument.type); + return std::make_unique>( + null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type, json_return_type); + } +}; + + +struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; +struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; }; +struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; +struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; +struct NameJSONType { static constexpr auto name{"JSONType"}; }; +struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; +struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; +struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; +struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; +struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; +struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; +struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; +struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; +struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; +struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; +struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; + + +template +class JSONHasImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) + { + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(1); + return true; + } +}; + + +template +class IsValidJSONImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() != 1) + { + /// IsValidJSON() shouldn't get parameters other than JSON. + throw Exception{"Function " + String(function_name) + " needs exactly one argument", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + } + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } + + static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) + { + /// This function is called only if JSON is valid. + /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(1); + return true; + } +}; + + +template +class JSONLengthImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + size_t size; + if (element.isArray()) + size = element.getArray().size(); + else if (element.isObject()) + size = element.getObject().size(); + else + return false; + + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(size); + return true; + } +}; + + +template +class JSONKeyImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view & last_key) + { + if (last_key.empty()) + return false; + ColumnString & col_str = assert_cast(dest); + col_str.insertData(last_key.data(), last_key.size()); + return true; + } +}; + + +template +class JSONTypeImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + static const std::vector> values = { + {"Array", '['}, + {"Object", '{'}, + {"String", '"'}, + {"Int64", 'i'}, + {"UInt64", 'u'}, + {"Double", 'd'}, + {"Bool", 'b'}, + {"Null", 0}, /// the default value for the column. + }; + return std::make_shared>(values); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + UInt8 type; + if (element.isInt64()) + type = 'i'; + else if (element.isUInt64()) + type = 'u'; + else if (element.isDouble()) + type = 'd'; + else if (element.isBool()) + type = 'b'; + else if (element.isString()) + type = '"'; + else if (element.isArray()) + type = '['; + else if (element.isObject()) + type = '{'; + else if (element.isNull()) + type = 0; + else + return false; + + ColumnVector & col_vec = assert_cast &>(dest); + col_vec.insertValue(type); + return true; + } +}; + + +template +class JSONExtractNumericImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared>(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + NumberType value; + + if (element.isInt64()) + { + if (!accurate::convertNumeric(element.getInt64(), value)) + return false; + } + else if (element.isUInt64()) + { + if (!accurate::convertNumeric(element.getUInt64(), value)) + return false; + } + else if (element.isDouble()) + { + if constexpr (std::is_floating_point_v) + { + /// We permit inaccurate conversion of double to float. + /// Example: double 0.1 from JSON is not representable in float. + /// But it will be more convenient for user to perform conversion. + value = element.getDouble(); + } + else if (!accurate::convertNumeric(element.getDouble(), value)) + return false; + } + else if (element.isBool() && is_integer && convert_bool_to_integer) + { + value = static_cast(element.getBool()); } else + return false; + + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(value); + return true; + } +}; + + +template +using JSONExtractInt8Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt8Impl = JSONExtractNumericImpl; +template +using JSONExtractInt16Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt16Impl = JSONExtractNumericImpl; +template +using JSONExtractInt32Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt32Impl = JSONExtractNumericImpl; +template +using JSONExtractInt64Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt64Impl = JSONExtractNumericImpl; +template +using JSONExtractFloat32Impl = JSONExtractNumericImpl; +template +using JSONExtractFloat64Impl = JSONExtractNumericImpl; + + +template +class JSONExtractBoolImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isBool()) + return false; + + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(static_cast(element.getBool())); + return true; + } +}; + + +template +class JSONExtractStringImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isString()) + return false; + + auto str = element.getString(); + ColumnString & col_str = assert_cast(dest); + col_str.insertData(str.data(), str.size()); + return true; + } +}; + +template +class JSONExtractRawImpl; + +/// Nodes of the extract tree. We need the extract tree to extract from JSON complex values containing array, tuples or nullables. +template +struct JSONExtractTree +{ + using Element = typename JSONParser::Element; + + class Node + { + public: + Node() = default; + virtual ~Node() = default; + virtual bool insertResultToColumn(IColumn &, const Element &) = 0; + }; + + template + class NumericNode : public Node + { + public: + bool insertResultToColumn(IColumn & dest, const Element & element) override { - if (isString(column.type)) - moves.emplace_back(MoveType::Key, ""); + return JSONExtractNumericImpl::insertResultToColumn(dest, element, {}); + } + }; + + class LowCardinalityNode : public Node + { + public: + LowCardinalityNode(DataTypePtr dictionary_type_, std::unique_ptr impl_) + : dictionary_type(dictionary_type_), impl(std::move(impl_)) {} + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + auto from_col = dictionary_type->createColumn(); + if (impl->insertResultToColumn(*from_col, element)) + { + StringRef value = from_col->getDataAt(0); + assert_cast(dest).insertData(value.data, value.size); + return true; + } + return false; + } + private: + DataTypePtr dictionary_type; + std::unique_ptr impl; + }; + + class UUIDNode : public Node + { + public: + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + if (!element.isString()) + return false; + + auto uuid = parseFromString(element.getString()); + assert_cast(dest).insert(uuid); + return true; + } + }; + + template + class DecimalNode : public Node + { + public: + DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + if (!element.isDouble()) + return false; + + const auto * type = assert_cast *>(data_type.get()); + auto result = convertToDecimal, DataTypeDecimal>(element.getDouble(), type->getScale()); + assert_cast &>(dest).insert(result); + return true; + } + private: + DataTypePtr data_type; + }; + + class StringNode : public Node + { + public: + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + if (element.isString()) + return JSONExtractStringImpl::insertResultToColumn(dest, element, {}); + else if (element.isNull()) + return false; else - moves.emplace_back(MoveType::Index, 0); + return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); + } + }; + + class FixedStringNode : public Node + { + public: + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + if (!element.isString()) + return false; + auto & col_str = assert_cast(dest); + auto str = element.getString(); + if (str.size() > col_str.getN()) + return false; + col_str.insertData(str.data(), str.size()); + return true; + } + }; + + template + class EnumNode : public Node + { + public: + EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) + { + for (const auto & name_value_pair : name_value_pairs) + { + name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); + only_values.emplace(name_value_pair.second); + } + } + + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + auto & col_vec = assert_cast &>(dest); + + if (element.isInt64()) + { + Type value; + if (!accurate::convertNumeric(element.getInt64(), value) || !only_values.count(value)) + return false; + col_vec.insertValue(value); + return true; + } + + if (element.isUInt64()) + { + Type value; + if (!accurate::convertNumeric(element.getUInt64(), value) || !only_values.count(value)) + return false; + col_vec.insertValue(value); + return true; + } + + if (element.isString()) + { + auto value = name_to_value_map.find(element.getString()); + if (value == name_to_value_map.end()) + return false; + col_vec.insertValue(value->second); + return true; + } + + return false; + } + + private: + std::vector> name_value_pairs; + std::unordered_map name_to_value_map; + std::unordered_set only_values; + }; + + class NullableNode : public Node + { + public: + NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} + + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + ColumnNullable & col_null = assert_cast(dest); + if (!nested->insertResultToColumn(col_null.getNestedColumn(), element)) + return false; + col_null.getNullMapColumn().insertValue(0); + return true; + } + + private: + std::unique_ptr nested; + }; + + class ArrayNode : public Node + { + public: + ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} + + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + if (!element.isArray()) + return false; + + auto array = element.getArray(); + + ColumnArray & col_arr = assert_cast(dest); + auto & data = col_arr.getData(); + size_t old_size = data.size(); + bool were_valid_elements = false; + + for (auto value : array) + { + if (nested->insertResultToColumn(data, value)) + were_valid_elements = true; + else + data.insertDefault(); + } + + if (!were_valid_elements) + { + data.popBack(data.size() - old_size); + return false; + } + + col_arr.getOffsets().push_back(data.size()); + return true; + } + + private: + std::unique_ptr nested; + }; + + class TupleNode : public Node + { + public: + TupleNode(std::vector> nested_, const std::vector & explicit_names_) : nested(std::move(nested_)), explicit_names(explicit_names_) + { + for (size_t i = 0; i != explicit_names.size(); ++i) + name_to_index_map.emplace(explicit_names[i], i); + } + + bool insertResultToColumn(IColumn & dest, const Element & element) override + { + ColumnTuple & tuple = assert_cast(dest); + size_t old_size = dest.size(); + bool were_valid_elements = false; + + auto set_size = [&](size_t size) + { + for (size_t i = 0; i != tuple.tupleSize(); ++i) + { + auto & col = tuple.getColumn(i); + if (col.size() != size) + { + if (col.size() > size) + col.popBack(col.size() - size); + else + while (col.size() < size) + col.insertDefault(); + } + } + }; + + if (element.isArray()) + { + auto array = element.getArray(); + auto it = array.begin(); + + for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) + { + if (nested[index]->insertResultToColumn(tuple.getColumn(index), *it++)) + were_valid_elements = true; + else + tuple.getColumn(index).insertDefault(); + } + + set_size(old_size + static_cast(were_valid_elements)); + return were_valid_elements; + } + + if (element.isObject()) + { + auto object = element.getObject(); + if (name_to_index_map.empty()) + { + auto it = object.begin(); + for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) + { + if (nested[index]->insertResultToColumn(tuple.getColumn(index), (*it++).second)) + were_valid_elements = true; + else + tuple.getColumn(index).insertDefault(); + } + } + else + { + for (const auto & [key, value] : object) + { + auto index = name_to_index_map.find(key); + if (index != name_to_index_map.end()) + { + if (nested[index->second]->insertResultToColumn(tuple.getColumn(index->second), value)) + were_valid_elements = true; + } + } + } + + set_size(old_size + static_cast(were_valid_elements)); + return were_valid_elements; + } + + return false; + } + + private: + std::vector> nested; + std::vector explicit_names; + std::unordered_map name_to_index_map; + }; + + static std::unique_ptr build(const char * function_name, const DataTypePtr & type) + { + switch (type->getTypeId()) + { + case TypeIndex::UInt8: return std::make_unique>(); + case TypeIndex::UInt16: return std::make_unique>(); + case TypeIndex::UInt32: return std::make_unique>(); + case TypeIndex::UInt64: return std::make_unique>(); + case TypeIndex::Int8: return std::make_unique>(); + case TypeIndex::Int16: return std::make_unique>(); + case TypeIndex::Int32: return std::make_unique>(); + case TypeIndex::Int64: return std::make_unique>(); + case TypeIndex::Float32: return std::make_unique>(); + case TypeIndex::Float64: return std::make_unique>(); + case TypeIndex::String: return std::make_unique(); + case TypeIndex::FixedString: return std::make_unique(); + case TypeIndex::UUID: return std::make_unique(); + case TypeIndex::LowCardinality: + { + auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); + auto impl = build(function_name, dictionary_type); + return std::make_unique(dictionary_type, std::move(impl)); + } + case TypeIndex::Decimal256: return std::make_unique>(type); + case TypeIndex::Decimal128: return std::make_unique>(type); + case TypeIndex::Decimal64: return std::make_unique>(type); + case TypeIndex::Decimal32: return std::make_unique>(type); + case TypeIndex::Enum8: + return std::make_unique>(static_cast(*type).getValues()); + case TypeIndex::Enum16: + return std::make_unique>(static_cast(*type).getValues()); + case TypeIndex::Nullable: + { + return std::make_unique(build(function_name, static_cast(*type).getNestedType())); + } + case TypeIndex::Array: + { + return std::make_unique(build(function_name, static_cast(*type).getNestedType())); + } + case TypeIndex::Tuple: + { + const auto & tuple = static_cast(*type); + const auto & tuple_elements = tuple.getElements(); + std::vector> elements; + for (const auto & tuple_element : tuple_elements) + elements.emplace_back(build(function_name, tuple_element)); + return std::make_unique(std::move(elements), tuple.haveExplicitNames() ? tuple.getElementNames() : Strings{}); + } + default: + throw Exception{"Function " + String(function_name) + " doesn't support the return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } } - return moves; -} +}; -size_t FunctionJSONHelpers::calculateMaxSize(const ColumnString::Offsets & offsets) + +template +class JSONExtractImpl { - size_t max_size = 0; - for (const auto i : collections::range(0, offsets.size())) - { - size_t size = offsets[i] - offsets[i - 1]; - if (max_size < size) - max_size = size; - } - if (max_size) - --max_size; - return max_size; -} +public: + using Element = typename JSONParser::Element; + static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() < 2) + throw Exception{"Function " + String(function_name) + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + const auto & col = arguments.back(); + auto col_type_const = typeid_cast(col.column.get()); + if (!col_type_const || !isString(col.type)) + throw Exception{"The last argument of function " + String(function_name) + + " should be a constant string specifying the return data type, illegal value: " + col.name, + ErrorCodes::ILLEGAL_COLUMN}; + + return DataTypeFactory::instance().get(col_type_const->getValue()); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } + + void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) + { + extract_tree = JSONExtractTree::build(function_name, result_type); + } + + bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + return extract_tree->insertResultToColumn(dest, element); + } + +protected: + std::unique_ptr::Node> extract_tree; +}; + + +template +class JSONExtractKeysAndValuesImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) + { + if (arguments.size() < 2) + throw Exception{"Function " + String(function_name) + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + const auto & col = arguments.back(); + auto col_type_const = typeid_cast(col.column.get()); + if (!col_type_const || !isString(col.type)) + throw Exception{"The last argument of function " + String(function_name) + + " should be a constant string specifying the values' data type, illegal value: " + col.name, + ErrorCodes::ILLEGAL_COLUMN}; + + DataTypePtr key_type = std::make_unique(); + DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); + DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); + return std::make_unique(tuple_type); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } + + void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) + { + const auto tuple_type = typeid_cast(result_type.get())->getNestedType(); + const auto value_type = typeid_cast(tuple_type.get())->getElements()[1]; + extract_tree = JSONExtractTree::build(function_name, value_type); + } + + bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); + size_t old_size = col_tuple.size(); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_value = col_tuple.getColumn(1); + + for (const auto & [key, value] : object) + { + if (extract_tree->insertResultToColumn(col_value, value)) + col_key.insertData(key.data(), key.size()); + } + + if (col_tuple.size() == old_size) + return false; + + col_arr.getOffsets().push_back(col_tuple.size()); + return true; + } + +private: + std::unique_ptr::Node> extract_tree; +}; + + +template +class JSONExtractRawImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + ColumnString & col_str = assert_cast(dest); + auto & chars = col_str.getChars(); + WriteBufferFromVector buf(chars, WriteBufferFromVector::AppendModeTag()); + traverse(element, buf); + buf.finalize(); + chars.push_back(0); + col_str.getOffsets().push_back(chars.size()); + return true; + } + +private: + static void traverse(const Element & element, WriteBuffer & buf) + { + if (element.isInt64()) + { + writeIntText(element.getInt64(), buf); + return; + } + if (element.isUInt64()) + { + writeIntText(element.getUInt64(), buf); + return; + } + if (element.isDouble()) + { + writeFloatText(element.getDouble(), buf); + return; + } + if (element.isBool()) + { + if (element.getBool()) + writeCString("true", buf); + else + writeCString("false", buf); + return; + } + if (element.isString()) + { + writeJSONString(element.getString(), buf, format_settings()); + return; + } + if (element.isArray()) + { + writeChar('[', buf); + bool need_comma = false; + for (auto value : element.getArray()) + { + if (std::exchange(need_comma, true)) + writeChar(',', buf); + traverse(value, buf); + } + writeChar(']', buf); + return; + } + if (element.isObject()) + { + writeChar('{', buf); + bool need_comma = false; + for (auto [key, value] : element.getObject()) + { + if (std::exchange(need_comma, true)) + writeChar(',', buf); + writeJSONString(key, buf, format_settings()); + writeChar(':', buf); + traverse(value, buf); + } + writeChar('}', buf); + return; + } + if (element.isNull()) + { + writeCString("null", buf); + return; + } + } + + static const FormatSettings & format_settings() + { + static const FormatSettings the_instance = [] + { + FormatSettings settings; + settings.json.escape_forward_slashes = false; + return settings; + }(); + return the_instance; + } +}; + + +template +class JSONExtractArrayRawImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_shared(std::make_shared()); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isArray()) + return false; + + auto array = element.getArray(); + ColumnArray & col_res = assert_cast(dest); + + for (auto value : array) + JSONExtractRawImpl::insertResultToColumn(col_res.getData(), value, {}); + + col_res.getOffsets().push_back(col_res.getOffsets().back() + array.size()); + return true; + } +}; + + +template +class JSONExtractKeysAndValuesRawImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + DataTypePtr string_type = std::make_unique(); + DataTypePtr tuple_type = std::make_unique(DataTypes{string_type, string_type}); + return std::make_unique(tuple_type); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + auto & col_arr = assert_cast(dest); + auto & col_tuple = assert_cast(col_arr.getData()); + auto & col_key = assert_cast(col_tuple.getColumn(0)); + auto & col_value = assert_cast(col_tuple.getColumn(1)); + + for (const auto & [key, value] : object) + { + col_key.insertData(key.data(), key.size()); + JSONExtractRawImpl::insertResultToColumn(col_value, value, {}); + } + + col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); + return true; + } +}; + +template +class JSONExtractKeysImpl +{ +public: + using Element = typename JSONParser::Element; + + static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) + { + return std::make_unique(std::make_shared()); + } + + static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } + + bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) + { + if (!element.isObject()) + return false; + + auto object = element.getObject(); + + ColumnArray & col_res = assert_cast(dest); + auto & col_key = assert_cast(col_res.getData()); + + for (const auto & [key, value] : object) + { + col_key.insertData(key.data(), key.size()); + } + + col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); + return true; + } +}; void registerFunctionsJSON(FunctionFactory & factory) { diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h deleted file mode 100644 index 7dd724dbfc9..00000000000 --- a/src/Functions/FunctionsJSON.h +++ /dev/null @@ -1,1390 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if !defined(ARCADIA_BUILD) -# include "config_functions.h" -#endif - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - - -/// Functions to parse JSONs and extract values from it. -/// The first argument of all these functions gets a JSON, -/// after that there are any number of arguments specifying path to a desired part from the JSON's root. -/// For example, -/// select JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) = -100 - -class FunctionJSONHelpers -{ -public: - template typename Impl, class JSONParser> - class Executor - { - public: - static ColumnPtr run(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) - { - MutableColumnPtr to{result_type->createColumn()}; - to->reserve(input_rows_count); - - if (arguments.empty()) - throw Exception{"Function " + String(Name::name) + " requires at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - const auto & first_column = arguments[0]; - if (!isString(first_column.type)) - throw Exception{"The first argument of function " + String(Name::name) + " should be a string containing JSON, illegal type: " + first_column.type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - const ColumnPtr & arg_json = first_column.column; - const auto * col_json_const = typeid_cast(arg_json.get()); - const auto * col_json_string - = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); - - if (!col_json_string) - throw Exception{"Illegal column " + arg_json->getName(), ErrorCodes::ILLEGAL_COLUMN}; - - const ColumnString::Chars & chars = col_json_string->getChars(); - const ColumnString::Offsets & offsets = col_json_string->getOffsets(); - - size_t num_index_arguments = Impl::getNumberOfIndexArguments(arguments); - std::vector moves = prepareMoves(Name::name, arguments, 1, num_index_arguments); - - /// Preallocate memory in parser if necessary. - JSONParser parser; - if constexpr (has_member_function_reserve::value) - { - size_t max_size = calculateMaxSize(offsets); - if (max_size) - parser.reserve(max_size); - } - - Impl impl; - - /// prepare() does Impl-specific preparation before handling each row. - if constexpr (has_member_function_prepare::*)(const char *, const ColumnsWithTypeAndName &, const DataTypePtr &)>::value) - impl.prepare(Name::name, arguments, result_type); - - using Element = typename JSONParser::Element; - - Element document; - bool document_ok = false; - if (col_json_const) - { - std::string_view json{reinterpret_cast(&chars[0]), offsets[0] - 1}; - document_ok = parser.parse(json, document); - } - - for (const auto i : collections::range(0, input_rows_count)) - { - if (!col_json_const) - { - std::string_view json{reinterpret_cast(&chars[offsets[i - 1]]), offsets[i] - offsets[i - 1] - 1}; - document_ok = parser.parse(json, document); - } - - bool added_to_column = false; - if (document_ok) - { - /// Perform moves. - Element element; - std::string_view last_key; - bool moves_ok = performMoves(arguments, i, document, moves, element, last_key); - - if (moves_ok) - added_to_column = impl.insertResultToColumn(*to, element, last_key); - } - - /// We add default value (=null or zero) if something goes wrong, we don't throw exceptions in these JSON functions. - if (!added_to_column) - to->insertDefault(); - } - return to; - } - }; - -private: - BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) - BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) - - template - struct has_index_operator : std::false_type {}; - - template - struct has_index_operator()[0])>> : std::true_type {}; - - /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. - /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) - /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. - /// Keys and indices can be nonconst, in this case they are calculated for each row. - enum class MoveType - { - Key, - Index, - ConstKey, - ConstIndex, - }; - - struct Move - { - Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} - Move(MoveType type_, const String & key_) : type(type_), key(key_) {} - MoveType type; - size_t index = 0; - String key; - }; - - static std::vector prepareMoves(const char * function_name, const ColumnsWithTypeAndName & columns, size_t first_index_argument, size_t num_index_arguments); - - /// Performs moves of types MoveType::Index and MoveType::ConstIndex. - template - static bool performMoves(const ColumnsWithTypeAndName & arguments, size_t row, - const typename JSONParser::Element & document, const std::vector & moves, - typename JSONParser::Element & element, std::string_view & last_key) - { - typename JSONParser::Element res_element = document; - std::string_view key; - - for (size_t j = 0; j != moves.size(); ++j) - { - switch (moves[j].type) - { - case MoveType::ConstIndex: - { - if (!moveToElementByIndex(res_element, moves[j].index, key)) - return false; - break; - } - case MoveType::ConstKey: - { - key = moves[j].key; - if (!moveToElementByKey(res_element, key)) - return false; - break; - } - case MoveType::Index: - { - Int64 index = (*arguments[j + 1].column)[row].get(); - if (!moveToElementByIndex(res_element, index, key)) - return false; - break; - } - case MoveType::Key: - { - key = std::string_view{(*arguments[j + 1].column).getDataAt(row)}; - if (!moveToElementByKey(res_element, key)) - return false; - break; - } - } - } - - element = res_element; - last_key = key; - return true; - } - - template - static bool moveToElementByIndex(typename JSONParser::Element & element, int index, std::string_view & out_key) - { - if (element.isArray()) - { - auto array = element.getArray(); - if (index >= 0) - --index; - else - index += array.size(); - - if (static_cast(index) >= array.size()) - return false; - element = array[index]; - out_key = {}; - return true; - } - - if constexpr (has_index_operator::value) - { - if (element.isObject()) - { - auto object = element.getObject(); - if (index >= 0) - --index; - else - index += object.size(); - - if (static_cast(index) >= object.size()) - return false; - std::tie(out_key, element) = object[index]; - return true; - } - } - - return {}; - } - - /// Performs moves of types MoveType::Key and MoveType::ConstKey. - template - static bool moveToElementByKey(typename JSONParser::Element & element, const std::string_view & key) - { - if (!element.isObject()) - return false; - auto object = element.getObject(); - return object.find(key, element); - } - - static size_t calculateMaxSize(const ColumnString::Offsets & offsets); -}; - - -template typename Impl> -class ExecutableFunctionJSON : public IExecutableFunction, WithContext -{ - -public: - explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_, const DataTypePtr & json_return_type_) - : null_presence(null_presence_), allow_simdjson(allow_simdjson_), json_return_type(json_return_type_) - { - } - - String getName() const override { return Name::name; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForConstants() const override { return true; } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override - { - if (null_presence.has_null_constant) - return result_type->createColumnConstWithDefaultValue(input_rows_count); - - ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; - ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, json_return_type, input_rows_count); - if (null_presence.has_nullable) - return wrapInNullable(temporary_result, arguments, result_type, input_rows_count); - return temporary_result; - } - -private: - - ColumnPtr - chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const - { -#if USE_SIMDJSON - if (allow_simdjson) - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -#endif - -#if USE_RAPIDJSON - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -#else - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); -#endif - } - - NullPresence null_presence; - bool allow_simdjson; - DataTypePtr json_return_type; -}; - - -template typename Impl> -class FunctionBaseFunctionJSON : public IFunctionBase -{ -public: - explicit FunctionBaseFunctionJSON( - const NullPresence & null_presence_, - bool allow_simdjson_, - DataTypes argument_types_, - DataTypePtr return_type_, - DataTypePtr json_return_type_) - : null_presence(null_presence_) - , allow_simdjson(allow_simdjson_) - , argument_types(std::move(argument_types_)) - , return_type(std::move(return_type_)) - , json_return_type(std::move(json_return_type_)) - { - } - - String getName() const override { return Name::name; } - - const DataTypes & getArgumentTypes() const override - { - return argument_types; - } - - const DataTypePtr & getResultType() const override - { - return return_type; - } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override - { - return std::make_unique>(null_presence, allow_simdjson, json_return_type); - } - -private: - NullPresence null_presence; - bool allow_simdjson; - DataTypes argument_types; - DataTypePtr return_type; - DataTypePtr json_return_type; -}; - - -/// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. -/// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. -template typename Impl> -class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext -{ -public: - static constexpr auto name = Name::name; - - String getName() const override { return name; } - - static FunctionOverloadResolverPtr create(ContextPtr context_) - { - return std::make_unique(context_); - } - - explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForNulls() const override { return false; } - - FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override - { - DataTypePtr json_return_type = Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments)); - NullPresence null_presence = getNullPresense(arguments); - DataTypePtr return_type; - if (null_presence.has_null_constant) - return_type = makeNullable(std::make_shared()); - else if (null_presence.has_nullable) - return_type = makeNullable(json_return_type); - else - return_type = json_return_type; - - /// Top-level LowCardinality columns are processed outside JSON parser. - json_return_type = removeLowCardinality(json_return_type); - - DataTypes argument_types; - argument_types.reserve(arguments.size()); - for (const auto & argument : arguments) - argument_types.emplace_back(argument.type); - return std::make_unique>( - null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type, json_return_type); - } -}; - - -struct NameJSONHas { static constexpr auto name{"JSONHas"}; }; -struct NameIsValidJSON { static constexpr auto name{"isValidJSON"}; }; -struct NameJSONLength { static constexpr auto name{"JSONLength"}; }; -struct NameJSONKey { static constexpr auto name{"JSONKey"}; }; -struct NameJSONType { static constexpr auto name{"JSONType"}; }; -struct NameJSONExtractInt { static constexpr auto name{"JSONExtractInt"}; }; -struct NameJSONExtractUInt { static constexpr auto name{"JSONExtractUInt"}; }; -struct NameJSONExtractFloat { static constexpr auto name{"JSONExtractFloat"}; }; -struct NameJSONExtractBool { static constexpr auto name{"JSONExtractBool"}; }; -struct NameJSONExtractString { static constexpr auto name{"JSONExtractString"}; }; -struct NameJSONExtract { static constexpr auto name{"JSONExtract"}; }; -struct NameJSONExtractKeysAndValues { static constexpr auto name{"JSONExtractKeysAndValues"}; }; -struct NameJSONExtractRaw { static constexpr auto name{"JSONExtractRaw"}; }; -struct NameJSONExtractArrayRaw { static constexpr auto name{"JSONExtractArrayRaw"}; }; -struct NameJSONExtractKeysAndValuesRaw { static constexpr auto name{"JSONExtractKeysAndValuesRaw"}; }; -struct NameJSONExtractKeys { static constexpr auto name{"JSONExtractKeys"}; }; - - -template -class JSONHasImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) { return std::make_shared(); } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) - { - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(1); - return true; - } -}; - - -template -class IsValidJSONImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() != 1) - { - /// IsValidJSON() shouldn't get parameters other than JSON. - throw Exception{"Function " + String(function_name) + " needs exactly one argument", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - } - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName &) { return 0; } - - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view &) - { - /// This function is called only if JSON is valid. - /// If JSON isn't valid then `FunctionJSON::Executor::run()` adds default value (=zero) to `dest` without calling this function. - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(1); - return true; - } -}; - - -template -class JSONLengthImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - size_t size; - if (element.isArray()) - size = element.getArray().size(); - else if (element.isObject()) - size = element.getObject().size(); - else - return false; - - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(size); - return true; - } -}; - - -template -class JSONKeyImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element &, const std::string_view & last_key) - { - if (last_key.empty()) - return false; - ColumnString & col_str = assert_cast(dest); - col_str.insertData(last_key.data(), last_key.size()); - return true; - } -}; - - -template -class JSONTypeImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - static const std::vector> values = { - {"Array", '['}, - {"Object", '{'}, - {"String", '"'}, - {"Int64", 'i'}, - {"UInt64", 'u'}, - {"Double", 'd'}, - {"Bool", 'b'}, - {"Null", 0}, /// the default value for the column. - }; - return std::make_shared>(values); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - UInt8 type; - if (element.isInt64()) - type = 'i'; - else if (element.isUInt64()) - type = 'u'; - else if (element.isDouble()) - type = 'd'; - else if (element.isBool()) - type = 'b'; - else if (element.isString()) - type = '"'; - else if (element.isArray()) - type = '['; - else if (element.isObject()) - type = '{'; - else if (element.isNull()) - type = 0; - else - return false; - - ColumnVector & col_vec = assert_cast &>(dest); - col_vec.insertValue(type); - return true; - } -}; - - -template -class JSONExtractNumericImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared>(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - NumberType value; - - if (element.isInt64()) - { - if (!accurate::convertNumeric(element.getInt64(), value)) - return false; - } - else if (element.isUInt64()) - { - if (!accurate::convertNumeric(element.getUInt64(), value)) - return false; - } - else if (element.isDouble()) - { - if constexpr (std::is_floating_point_v) - { - /// We permit inaccurate conversion of double to float. - /// Example: double 0.1 from JSON is not representable in float. - /// But it will be more convenient for user to perform conversion. - value = element.getDouble(); - } - else if (!accurate::convertNumeric(element.getDouble(), value)) - return false; - } - else if (element.isBool() && is_integer && convert_bool_to_integer) - { - value = static_cast(element.getBool()); - } - else - return false; - - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(value); - return true; - } -}; - - -template -using JSONExtractInt8Impl = JSONExtractNumericImpl; -template -using JSONExtractUInt8Impl = JSONExtractNumericImpl; -template -using JSONExtractInt16Impl = JSONExtractNumericImpl; -template -using JSONExtractUInt16Impl = JSONExtractNumericImpl; -template -using JSONExtractInt32Impl = JSONExtractNumericImpl; -template -using JSONExtractUInt32Impl = JSONExtractNumericImpl; -template -using JSONExtractInt64Impl = JSONExtractNumericImpl; -template -using JSONExtractUInt64Impl = JSONExtractNumericImpl; -template -using JSONExtractFloat32Impl = JSONExtractNumericImpl; -template -using JSONExtractFloat64Impl = JSONExtractNumericImpl; - - -template -class JSONExtractBoolImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - if (!element.isBool()) - return false; - - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(static_cast(element.getBool())); - return true; - } -}; - - -template -class JSONExtractStringImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - if (!element.isString()) - return false; - - auto str = element.getString(); - ColumnString & col_str = assert_cast(dest); - col_str.insertData(str.data(), str.size()); - return true; - } -}; - -template -class JSONExtractRawImpl; - -/// Nodes of the extract tree. We need the extract tree to extract from JSON complex values containing array, tuples or nullables. -template -struct JSONExtractTree -{ - using Element = typename JSONParser::Element; - - class Node - { - public: - Node() = default; - virtual ~Node() = default; - virtual bool insertResultToColumn(IColumn &, const Element &) = 0; - }; - - template - class NumericNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - return JSONExtractNumericImpl::insertResultToColumn(dest, element, {}); - } - }; - - class LowCardinalityNode : public Node - { - public: - LowCardinalityNode(DataTypePtr dictionary_type_, std::unique_ptr impl_) - : dictionary_type(dictionary_type_), impl(std::move(impl_)) {} - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - auto from_col = dictionary_type->createColumn(); - if (impl->insertResultToColumn(*from_col, element)) - { - StringRef value = from_col->getDataAt(0); - assert_cast(dest).insertData(value.data, value.size); - return true; - } - return false; - } - private: - DataTypePtr dictionary_type; - std::unique_ptr impl; - }; - - class UUIDNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isString()) - return false; - - auto uuid = parseFromString(element.getString()); - assert_cast(dest).insert(uuid); - return true; - } - }; - - template - class DecimalNode : public Node - { - public: - DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isDouble()) - return false; - - const auto * type = assert_cast *>(data_type.get()); - auto result = convertToDecimal, DataTypeDecimal>(element.getDouble(), type->getScale()); - assert_cast &>(dest).insert(result); - return true; - } - private: - DataTypePtr data_type; - }; - - class StringNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (element.isString()) - return JSONExtractStringImpl::insertResultToColumn(dest, element, {}); - else if (element.isNull()) - return false; - else - return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); - } - }; - - class FixedStringNode : public Node - { - public: - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isString()) - return false; - auto & col_str = assert_cast(dest); - auto str = element.getString(); - if (str.size() > col_str.getN()) - return false; - col_str.insertData(str.data(), str.size()); - return true; - } - }; - - template - class EnumNode : public Node - { - public: - EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) - { - for (const auto & name_value_pair : name_value_pairs) - { - name_to_value_map.emplace(name_value_pair.first, name_value_pair.second); - only_values.emplace(name_value_pair.second); - } - } - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - auto & col_vec = assert_cast &>(dest); - - if (element.isInt64()) - { - Type value; - if (!accurate::convertNumeric(element.getInt64(), value) || !only_values.count(value)) - return false; - col_vec.insertValue(value); - return true; - } - - if (element.isUInt64()) - { - Type value; - if (!accurate::convertNumeric(element.getUInt64(), value) || !only_values.count(value)) - return false; - col_vec.insertValue(value); - return true; - } - - if (element.isString()) - { - auto value = name_to_value_map.find(element.getString()); - if (value == name_to_value_map.end()) - return false; - col_vec.insertValue(value->second); - return true; - } - - return false; - } - - private: - std::vector> name_value_pairs; - std::unordered_map name_to_value_map; - std::unordered_set only_values; - }; - - class NullableNode : public Node - { - public: - NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - ColumnNullable & col_null = assert_cast(dest); - if (!nested->insertResultToColumn(col_null.getNestedColumn(), element)) - return false; - col_null.getNullMapColumn().insertValue(0); - return true; - } - - private: - std::unique_ptr nested; - }; - - class ArrayNode : public Node - { - public: - ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - if (!element.isArray()) - return false; - - auto array = element.getArray(); - - ColumnArray & col_arr = assert_cast(dest); - auto & data = col_arr.getData(); - size_t old_size = data.size(); - bool were_valid_elements = false; - - for (auto value : array) - { - if (nested->insertResultToColumn(data, value)) - were_valid_elements = true; - else - data.insertDefault(); - } - - if (!were_valid_elements) - { - data.popBack(data.size() - old_size); - return false; - } - - col_arr.getOffsets().push_back(data.size()); - return true; - } - - private: - std::unique_ptr nested; - }; - - class TupleNode : public Node - { - public: - TupleNode(std::vector> nested_, const std::vector & explicit_names_) : nested(std::move(nested_)), explicit_names(explicit_names_) - { - for (size_t i = 0; i != explicit_names.size(); ++i) - name_to_index_map.emplace(explicit_names[i], i); - } - - bool insertResultToColumn(IColumn & dest, const Element & element) override - { - ColumnTuple & tuple = assert_cast(dest); - size_t old_size = dest.size(); - bool were_valid_elements = false; - - auto set_size = [&](size_t size) - { - for (size_t i = 0; i != tuple.tupleSize(); ++i) - { - auto & col = tuple.getColumn(i); - if (col.size() != size) - { - if (col.size() > size) - col.popBack(col.size() - size); - else - while (col.size() < size) - col.insertDefault(); - } - } - }; - - if (element.isArray()) - { - auto array = element.getArray(); - auto it = array.begin(); - - for (size_t index = 0; (index != nested.size()) && (it != array.end()); ++index) - { - if (nested[index]->insertResultToColumn(tuple.getColumn(index), *it++)) - were_valid_elements = true; - else - tuple.getColumn(index).insertDefault(); - } - - set_size(old_size + static_cast(were_valid_elements)); - return were_valid_elements; - } - - if (element.isObject()) - { - auto object = element.getObject(); - if (name_to_index_map.empty()) - { - auto it = object.begin(); - for (size_t index = 0; (index != nested.size()) && (it != object.end()); ++index) - { - if (nested[index]->insertResultToColumn(tuple.getColumn(index), (*it++).second)) - were_valid_elements = true; - else - tuple.getColumn(index).insertDefault(); - } - } - else - { - for (auto & [key, value] : object) - { - auto index = name_to_index_map.find(key); - if (index != name_to_index_map.end()) - { - if (nested[index->second]->insertResultToColumn(tuple.getColumn(index->second), value)) - were_valid_elements = true; - } - } - } - - set_size(old_size + static_cast(were_valid_elements)); - return were_valid_elements; - } - - return false; - } - - private: - std::vector> nested; - std::vector explicit_names; - std::unordered_map name_to_index_map; - }; - - static std::unique_ptr build(const char * function_name, const DataTypePtr & type) - { - switch (type->getTypeId()) - { - case TypeIndex::UInt8: return std::make_unique>(); - case TypeIndex::UInt16: return std::make_unique>(); - case TypeIndex::UInt32: return std::make_unique>(); - case TypeIndex::UInt64: return std::make_unique>(); - case TypeIndex::Int8: return std::make_unique>(); - case TypeIndex::Int16: return std::make_unique>(); - case TypeIndex::Int32: return std::make_unique>(); - case TypeIndex::Int64: return std::make_unique>(); - case TypeIndex::Float32: return std::make_unique>(); - case TypeIndex::Float64: return std::make_unique>(); - case TypeIndex::String: return std::make_unique(); - case TypeIndex::FixedString: return std::make_unique(); - case TypeIndex::UUID: return std::make_unique(); - case TypeIndex::LowCardinality: - { - auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); - auto impl = build(function_name, dictionary_type); - return std::make_unique(dictionary_type, std::move(impl)); - } - case TypeIndex::Decimal256: return std::make_unique>(type); - case TypeIndex::Decimal128: return std::make_unique>(type); - case TypeIndex::Decimal64: return std::make_unique>(type); - case TypeIndex::Decimal32: return std::make_unique>(type); - case TypeIndex::Enum8: - return std::make_unique>(static_cast(*type).getValues()); - case TypeIndex::Enum16: - return std::make_unique>(static_cast(*type).getValues()); - case TypeIndex::Nullable: - { - return std::make_unique(build(function_name, static_cast(*type).getNestedType())); - } - case TypeIndex::Array: - { - return std::make_unique(build(function_name, static_cast(*type).getNestedType())); - } - case TypeIndex::Tuple: - { - const auto & tuple = static_cast(*type); - const auto & tuple_elements = tuple.getElements(); - std::vector> elements; - for (const auto & tuple_element : tuple_elements) - elements.emplace_back(build(function_name, tuple_element)); - return std::make_unique(std::move(elements), tuple.haveExplicitNames() ? tuple.getElementNames() : Strings{}); - } - default: - throw Exception{"Function " + String(function_name) + " doesn't support the return type schema: " + type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } - } -}; - - -template -class JSONExtractImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() < 2) - throw Exception{"Function " + String(function_name) + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - const auto & col = arguments.back(); - auto col_type_const = typeid_cast(col.column.get()); - if (!col_type_const || !isString(col.type)) - throw Exception{"The last argument of function " + String(function_name) - + " should be a constant string specifying the return data type, illegal value: " + col.name, - ErrorCodes::ILLEGAL_COLUMN}; - - return DataTypeFactory::instance().get(col_type_const->getValue()); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } - - void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) - { - extract_tree = JSONExtractTree::build(function_name, result_type); - } - - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - return extract_tree->insertResultToColumn(dest, element); - } - -protected: - std::unique_ptr::Node> extract_tree; -}; - - -template -class JSONExtractKeysAndValuesImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char * function_name, const ColumnsWithTypeAndName & arguments) - { - if (arguments.size() < 2) - throw Exception{"Function " + String(function_name) + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; - - const auto & col = arguments.back(); - auto col_type_const = typeid_cast(col.column.get()); - if (!col_type_const || !isString(col.type)) - throw Exception{"The last argument of function " + String(function_name) - + " should be a constant string specifying the values' data type, illegal value: " + col.name, - ErrorCodes::ILLEGAL_COLUMN}; - - DataTypePtr key_type = std::make_unique(); - DataTypePtr value_type = DataTypeFactory::instance().get(col_type_const->getValue()); - DataTypePtr tuple_type = std::make_unique(DataTypes{key_type, value_type}); - return std::make_unique(tuple_type); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 2; } - - void prepare(const char * function_name, const ColumnsWithTypeAndName &, const DataTypePtr & result_type) - { - const auto tuple_type = typeid_cast(result_type.get())->getNestedType(); - const auto value_type = typeid_cast(tuple_type.get())->getElements()[1]; - extract_tree = JSONExtractTree::build(function_name, value_type); - } - - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - if (!element.isObject()) - return false; - - auto object = element.getObject(); - - auto & col_arr = assert_cast(dest); - auto & col_tuple = assert_cast(col_arr.getData()); - size_t old_size = col_tuple.size(); - auto & col_key = assert_cast(col_tuple.getColumn(0)); - auto & col_value = col_tuple.getColumn(1); - - for (auto & [key, value] : object) - { - if (extract_tree->insertResultToColumn(col_value, value)) - col_key.insertData(key.data(), key.size()); - } - - if (col_tuple.size() == old_size) - return false; - - col_arr.getOffsets().push_back(col_tuple.size()); - return true; - } - -private: - std::unique_ptr::Node> extract_tree; -}; - - -template -class JSONExtractRawImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - ColumnString & col_str = assert_cast(dest); - auto & chars = col_str.getChars(); - WriteBufferFromVector buf(chars, WriteBufferFromVector::AppendModeTag()); - traverse(element, buf); - buf.finalize(); - chars.push_back(0); - col_str.getOffsets().push_back(chars.size()); - return true; - } - -private: - static void traverse(const Element & element, WriteBuffer & buf) - { - if (element.isInt64()) - { - writeIntText(element.getInt64(), buf); - return; - } - if (element.isUInt64()) - { - writeIntText(element.getUInt64(), buf); - return; - } - if (element.isDouble()) - { - writeFloatText(element.getDouble(), buf); - return; - } - if (element.isBool()) - { - if (element.getBool()) - writeCString("true", buf); - else - writeCString("false", buf); - return; - } - if (element.isString()) - { - writeJSONString(element.getString(), buf, format_settings()); - return; - } - if (element.isArray()) - { - writeChar('[', buf); - bool need_comma = false; - for (auto value : element.getArray()) - { - if (std::exchange(need_comma, true)) - writeChar(',', buf); - traverse(value, buf); - } - writeChar(']', buf); - return; - } - if (element.isObject()) - { - writeChar('{', buf); - bool need_comma = false; - for (auto [key, value] : element.getObject()) - { - if (std::exchange(need_comma, true)) - writeChar(',', buf); - writeJSONString(key, buf, format_settings()); - writeChar(':', buf); - traverse(value, buf); - } - writeChar('}', buf); - return; - } - if (element.isNull()) - { - writeCString("null", buf); - return; - } - } - - static const FormatSettings & format_settings() - { - static const FormatSettings the_instance = [] - { - FormatSettings settings; - settings.json.escape_forward_slashes = false; - return settings; - }(); - return the_instance; - } -}; - - -template -class JSONExtractArrayRawImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_shared(std::make_shared()); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - if (!element.isArray()) - return false; - - auto array = element.getArray(); - ColumnArray & col_res = assert_cast(dest); - - for (auto value : array) - JSONExtractRawImpl::insertResultToColumn(col_res.getData(), value, {}); - - col_res.getOffsets().push_back(col_res.getOffsets().back() + array.size()); - return true; - } -}; - - -template -class JSONExtractKeysAndValuesRawImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - DataTypePtr string_type = std::make_unique(); - DataTypePtr tuple_type = std::make_unique(DataTypes{string_type, string_type}); - return std::make_unique(tuple_type); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - if (!element.isObject()) - return false; - - auto object = element.getObject(); - - auto & col_arr = assert_cast(dest); - auto & col_tuple = assert_cast(col_arr.getData()); - auto & col_key = assert_cast(col_tuple.getColumn(0)); - auto & col_value = assert_cast(col_tuple.getColumn(1)); - - for (auto & [key, value] : object) - { - col_key.insertData(key.data(), key.size()); - JSONExtractRawImpl::insertResultToColumn(col_value, value, {}); - } - - col_arr.getOffsets().push_back(col_arr.getOffsets().back() + object.size()); - return true; - } -}; - -template -class JSONExtractKeysImpl -{ -public: - using Element = typename JSONParser::Element; - - static DataTypePtr getReturnType(const char *, const ColumnsWithTypeAndName &) - { - return std::make_unique(std::make_shared()); - } - - static size_t getNumberOfIndexArguments(const ColumnsWithTypeAndName & arguments) { return arguments.size() - 1; } - - bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) - { - if (!element.isObject()) - return false; - - auto object = element.getObject(); - - ColumnArray & col_res = assert_cast(dest); - auto & col_key = assert_cast(col_res.getData()); - - for (const auto & [key, value] : object) - { - col_key.insertData(key.data(), key.size()); - } - - col_res.getOffsets().push_back(col_res.getOffsets().back() + object.size()); - return true; - } -}; - -} From a51ab78c680b743a2834f3755ddfb68766ea43ce Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 14 Oct 2021 23:13:41 +0300 Subject: [PATCH 680/950] Fixed style check --- ..._add_database.sql => 02098_default_dict_get_add_database.sql} | 1 + 1 file changed, 1 insertion(+) rename tests/queries/0_stateless/{02097_default_dict_get_add_database.sql => 02098_default_dict_get_add_database.sql} (99%) mode change 100755 => 100644 diff --git a/tests/queries/0_stateless/02097_default_dict_get_add_database.sql b/tests/queries/0_stateless/02098_default_dict_get_add_database.sql old mode 100755 new mode 100644 similarity index 99% rename from tests/queries/0_stateless/02097_default_dict_get_add_database.sql rename to tests/queries/0_stateless/02098_default_dict_get_add_database.sql index d3e4bb6752d..af177566476 --- a/tests/queries/0_stateless/02097_default_dict_get_add_database.sql +++ b/tests/queries/0_stateless/02098_default_dict_get_add_database.sql @@ -41,3 +41,4 @@ DROP TABLE test_table; DROP TABLE test_table_default; DROP DATABASE 02097_db; + From bb8105951f651a6dec92c6546d7a4152b5ddb336 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:43:38 +0300 Subject: [PATCH 681/950] Disable fsync_metadata on CI Sometimes we see that pretty simple queries like CREATE DATABASE can take > 30 seconds, let's try to disable fsync. CI: https://clickhouse-test-reports.s3.yandex.net/30065/e5bc573250d3d6938937739b05d6f8cf618722db/functional_stateless_tests_(address).html#fail1 CI: https://clickhouse-test-reports.s3.yandex.net/30065/e5bc573250d3d6938937739b05d6f8cf618722db/functional_stateless_tests_(release).html#fail1 --- tests/config/install.sh | 1 + tests/config/users.d/no_fsync_metadata.xml | 11 +++++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/config/users.d/no_fsync_metadata.xml diff --git a/tests/config/install.sh b/tests/config/install.sh index 94ad55504a8..ba6ba0cd07c 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -48,6 +48,7 @@ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/no_fsync_metadata.xml b/tests/config/users.d/no_fsync_metadata.xml new file mode 100644 index 00000000000..6b9d69927f8 --- /dev/null +++ b/tests/config/users.d/no_fsync_metadata.xml @@ -0,0 +1,11 @@ + + + + + false + + + From 8ec43b7c702079fd134903293e1be86248c3d2af Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Fri, 15 Oct 2021 00:56:21 +0300 Subject: [PATCH 682/950] fixed type-conversion-functions en-ru --- .../functions/type-conversion-functions.md | 12 +++++++----- .../functions/type-conversion-functions.md | 16 +++++++++------- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index c36861f280a..88e1cf47592 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -281,22 +281,24 @@ Result: ## toDate32OrDefault {#todate32-or-default} -Converts the argument to the [Date32](../../sql-reference/data-types/date32.md) data type. If the value is outside the range returns the border values supported by `Date32`. If the argument has [Date](../../sql-reference/data-types/date.md) type, borders of `Date` are taken into account. Returns default value if invalid argument is received. +Converts the argument to the [Date32](../../sql-reference/data-types/date32.md) data type. If the value is outside the range returns the lower border value supported by `Date32`. If the argument has [Date](../../sql-reference/data-types/date.md) type, borders of `Date` are taken into account. Returns default value if invalid argument is received. **Example** Query: ``` sql -SELECT toDate32OrDefault('1955-01-01'), toDate32OrDefault(''); +SELECT + toDate32OrDefault('1930-01-01', toDate32('2020-01-01')), + toDate32OrDefault('xx1930-01-01', toDate32('2020-01-01')); ``` Result: ``` text -┌─toDate32OrDefault('1955-01-01')─┬─toDate32OrDefault('')─┐ -│ 1955-01-01 │ 1970-01-01 │ -└─────────────────────────────────┴───────────────────────┘ +┌─toDate32OrDefault('1930-01-01', toDate32('2020-01-01'))─┬─toDate32OrDefault('xx1930-01-01', toDate32('2020-01-01'))─┐ +│ 1930-01-01 │ 2020-01-01 │ +└─────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ ``` ## toDecimal(32\|64\|128\|256) {#todecimal3264128256} diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index c347b8ca346..1b4ea4ef609 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -280,22 +280,24 @@ SELECT toDate32OrNull('1955-01-01'), toDate32OrNull(''); ## toDate32OrDefault {#todate32-or-default} -Конвертирует аргумент в значение типа [Date32](../../sql-reference/data-types/date32.md). Если значение выходит за границы диапазона, возвращается пограничное значение `Date32`. Если аргумент имеет тип [Date](../../sql-reference/data-types/date.md), учитываются границы типа `Date`. Возвращает значение по умолчанию, если получен недопустимый аргумент. +Конвертирует аргумент в значение типа [Date32](../../sql-reference/data-types/date32.md). Если значение выходит за границы диапазона, возвращается нижнее пограничное значение `Date32`. Если аргумент имеет тип [Date](../../sql-reference/data-types/date.md), учитываются границы типа `Date`. Возвращает значение по умолчанию, если получен недопустимый аргумент. **Пример** Запрос: ``` sql -SELECT toDate32OrDefault('1955-01-01'), toDate32OrDefault(''); +SELECT + toDate32OrDefault('1930-01-01', toDate32('2020-01-01')), + toDate32OrDefault('xx1930-01-01', toDate32('2020-01-01')); ``` Результат: ``` text -┌─toDate32OrDefault('1955-01-01')─┬─toDate32OrDefault('')─┐ -│ 1955-01-01 │ 1970-01-01 │ -└─────────────────────────────────┴───────────────────────┘ +┌─toDate32OrDefault('1930-01-01', toDate32('2020-01-01'))─┬─toDate32OrDefault('xx1930-01-01', toDate32('2020-01-01'))─┐ +│ 1930-01-01 │ 2020-01-01 │ +└─────────────────────────────────────────────────────────┴───────────────────────────────────────────────────────────┘ ``` ## toDecimal(32\|64\|128\|256) {#todecimal3264128} @@ -378,10 +380,10 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); **Возвращаемое значение** -Значение типа `Nullable(Decimal(P,S))`. Значение содержит: +Значение типа `Decimal(P,S)`. Значение содержит: - Число с `S` десятичными знаками, если ClickHouse распознал число во входной строке. -- `NULL`, если ClickHouse не смог распознать число во входной строке или входное число содержит больше чем `S` десятичных знаков. +- Значение по умолчанию типа `Decimal(P,S)`, если ClickHouse не смог распознать число во входной строке или входное число содержит больше чем `S` десятичных знаков. **Примеры** From a063097fdffc7ca9e9a4bad859dbddc5fa926220 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:02:26 +0300 Subject: [PATCH 683/950] Fix INSERT SELECT incorrectly fills MATERIALIZED column based of Nullable column Required columns of the default expression should not be converted to NULL, since this map value to default and MATERIALIZED values will not work. Consider the following structure: - A Nullable(Int64) - X Int64 materialized coalesce(A, -1) With recursive_null_as_default=true you will get: _CAST(coalesce(A, -1), 'Int64') AS X, NULL AS A And this will ignore default expression. Fixes: #23524 (Cc: @kssenii) Fixes: #29729 (Cc: @tavplubix) Backport: 21.7+ --- src/Interpreters/inplaceBlockConversions.cpp | 20 ++++++++++++++++++- ...02053_INSERT_SELECT_MATERIALIZED.reference | 2 ++ .../02053_INSERT_SELECT_MATERIALIZED.sql | 6 ++++++ 3 files changed, 27 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.reference create mode 100644 tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.sql diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 4ba924821d7..2841abe757e 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -61,7 +61,25 @@ void addDefaultRequiredExpressionsRecursively( added_columns.emplace(required_column_name); for (const auto & next_required_column_name : required_columns_names) - addDefaultRequiredExpressionsRecursively(block, next_required_column_name, required_column_type, columns, default_expr_list_accum, added_columns, null_as_default); + { + /// Required columns of the default expression should not be converted to NULL, + /// since this map value to default and MATERIALIZED values will not work. + /// + /// Consider the following structure: + /// - A Nullable(Int64) + /// - X Int64 materialized coalesce(A, -1) + /// + /// With recursive_null_as_default=true you will get: + /// + /// _CAST(coalesce(A, -1), 'Int64') AS X, NULL AS A + /// + /// And this will ignore default expression. + bool recursive_null_as_default = false; + addDefaultRequiredExpressionsRecursively(block, + next_required_column_name, required_column_type, + columns, default_expr_list_accum, added_columns, + recursive_null_as_default); + } } else if (columns.has(required_column_name)) { diff --git a/tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.reference b/tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.reference new file mode 100644 index 00000000000..5154881396a --- /dev/null +++ b/tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.reference @@ -0,0 +1,2 @@ +1 42 42 +1 42 42 diff --git a/tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.sql b/tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.sql new file mode 100644 index 00000000000..e9ea0c9f016 --- /dev/null +++ b/tests/queries/0_stateless/02053_INSERT_SELECT_MATERIALIZED.sql @@ -0,0 +1,6 @@ +-- Test from https://github.com/ClickHouse/ClickHouse/issues/29729 +create table data_02053 (id Int64, A Nullable(Int64), X Int64 materialized coalesce(A, -1)) engine=MergeTree order by id; +insert into data_02053 values (1, 42); +-- Due to insert_null_as_default A became Null and X became -1 +insert into data_02053 select 1, 42; +select *, X from data_02053 order by id; From 5d5f94f456b83e27f11f2e3080a95679fa0edeb8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 20:41:40 +0300 Subject: [PATCH 684/950] Mark 01200_mutations_memory_consumption as long and allow run in fasttest --- tests/queries/bugs/01200_mutations_memory_consumption.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/bugs/01200_mutations_memory_consumption.sql b/tests/queries/bugs/01200_mutations_memory_consumption.sql index d3eb5dd165c..de9c2df7f08 100644 --- a/tests/queries/bugs/01200_mutations_memory_consumption.sql +++ b/tests/queries/bugs/01200_mutations_memory_consumption.sql @@ -1,4 +1,4 @@ --- Tags: no-debug, no-parallel, no-fasttest +-- Tags: no-debug, no-parallel, long DROP TABLE IF EXISTS table_with_single_pk; From ae980fcd622e50e6f85c8d6c69954a76dd2b2c9b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 Oct 2021 16:49:44 +0300 Subject: [PATCH 685/950] Move 01200_mutations_memory_consumption out from bugs --- .../01200_mutations_memory_consumption.reference | 0 .../{bugs => 0_stateless}/01200_mutations_memory_consumption.sql | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/{bugs => 0_stateless}/01200_mutations_memory_consumption.reference (100%) rename tests/queries/{bugs => 0_stateless}/01200_mutations_memory_consumption.sql (100%) diff --git a/tests/queries/bugs/01200_mutations_memory_consumption.reference b/tests/queries/0_stateless/01200_mutations_memory_consumption.reference similarity index 100% rename from tests/queries/bugs/01200_mutations_memory_consumption.reference rename to tests/queries/0_stateless/01200_mutations_memory_consumption.reference diff --git a/tests/queries/bugs/01200_mutations_memory_consumption.sql b/tests/queries/0_stateless/01200_mutations_memory_consumption.sql similarity index 100% rename from tests/queries/bugs/01200_mutations_memory_consumption.sql rename to tests/queries/0_stateless/01200_mutations_memory_consumption.sql From e4b39ca99cc16ce2f27faf5c2f30bf79cec43c84 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 6 Oct 2021 00:10:12 +0300 Subject: [PATCH 686/950] Remove unused Block member from MutateTask v0: Fix excessive memory usage in MutateTask This is relevant only for 01200_mutations_memory_consumption test, since this is just a block. v2: just remove unused Block member, since other part had been fixed in #29768 --- src/Storages/MergeTree/MutateTask.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 66d0f5f7e49..dbf05e1c6cc 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -766,7 +766,6 @@ private: State state{State::NEED_PREPARE}; MutationContextPtr ctx; - Block block; size_t block_num = 0; using ProjectionNameToItsBlocks = std::map; From fd38cbb0dfb10fc1211dba1404cd78dce29bd0f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 4 Oct 2021 01:41:35 +0300 Subject: [PATCH 687/950] Fix memory tracking for merges and mutations (by destroying earlier) It fixes only some tiny allocations, and so it should not affect any huge mutations/merges. And plus, this should not be a real fix, since peak_memory_usage is obtained before even destrying this objects, and destroing objects will unlikely update peak memory usage (although it is possible). v0: do this in dtors v2: do this earlier --- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 1 + src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 1 + src/Storages/MergeTree/MutateTask.cpp | 1 + 3 files changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 3129ae9aa35..af54bf113d6 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -87,6 +87,7 @@ void MergePlainMergeTreeTask::prepare() write_part_log = [this] (const ExecutionStatus & execution_status) { + merge_task.reset(); storage.writePartLog( PartLogElement::MERGE_PARTS, execution_status, diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 1f4b387a3a3..39a057d68ca 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -40,6 +40,7 @@ void MutatePlainMergeTreeTask::prepare() write_part_log = [this] (const ExecutionStatus & execution_status) { + mutate_task.reset(); storage.writePartLog( PartLogElement::MUTATE_PART, execution_status, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index dbf05e1c6cc..40037c38779 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -985,6 +985,7 @@ private: ctx->mutating_pipeline.reset(); static_pointer_cast(ctx->out)->writeSuffixAndFinalizePart(ctx->new_data_part, ctx->need_sync); + ctx->out.reset(); } enum class State From 8cc45bea7b8586e0d2e5133051169c96152ee5f0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:47:28 +0300 Subject: [PATCH 688/950] Avoid accounting memory from another mutation/merge Before this path it was possible that one merge/mutation may account memory from another, due to ThreadStatus::untracked_memory. And this cause flakiness of 01200_mutations_memory_consumption. --- src/Storages/MergeTree/MergeList.cpp | 6 ++++++ src/Storages/MergeTree/MergeList.h | 1 + 2 files changed, 7 insertions(+) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 07aabf64dfd..8e85bcb986f 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -34,6 +34,11 @@ MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MemoryTracker * memory_ prev_untracked_memory_limit = current_thread->untracked_memory_limit; current_thread->untracked_memory_limit = untracked_memory_limit; + + /// Avoid accounting memory from another mutation/merge + /// (NOTE: consider moving such code to ThreadFromGlobalPool and related places) + prev_untracked_memory = current_thread->untracked_memory; + current_thread->untracked_memory = 0; } @@ -45,6 +50,7 @@ MemoryTrackerThreadSwitcher::~MemoryTrackerThreadSwitcher() background_thread_memory_tracker->setParent(background_thread_memory_tracker_prev_parent); current_thread->untracked_memory_limit = prev_untracked_memory_limit; + current_thread->untracked_memory = prev_untracked_memory; } MergeListElement::MergeListElement( diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 95e96b5a73e..0389736a7f9 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -68,6 +68,7 @@ private: MemoryTracker * background_thread_memory_tracker; MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; UInt64 prev_untracked_memory_limit; + UInt64 prev_untracked_memory; }; using MemoryTrackerThreadSwitcherPtr = std::unique_ptr; From 3477ab52a75202879f912176c2e03f58f595a9fa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:47:28 +0300 Subject: [PATCH 689/950] Add StorageID::getShortName() --- src/Interpreters/StorageID.cpp | 2 +- src/Interpreters/StorageID.h | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/StorageID.cpp b/src/Interpreters/StorageID.cpp index bfb969cab25..2e76618e4c0 100644 --- a/src/Interpreters/StorageID.cpp +++ b/src/Interpreters/StorageID.cpp @@ -110,7 +110,7 @@ StorageID StorageID::fromDictionaryConfig(const Poco::Util::AbstractConfiguratio return res; } -String StorageID::getInternalDictionaryName() const +String StorageID::getShortName() const { assertNotEmpty(); if (hasUUID()) diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 623f481fbc5..f1fcfde25c0 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -91,7 +91,9 @@ struct StorageID /// If dictionary has UUID, then use it as dictionary name in ExternalLoader to allow dictionary renaming. /// ExternalDictnariesLoader::resolveDictionaryName(...) should be used to access such dictionaries by name. - String getInternalDictionaryName() const; + String getInternalDictionaryName() const { return getShortName(); } + /// Get short, but unique, name. + String getShortName() const; private: StorageID() = default; From 8a209a78d78aae63ffb1089bc8b442132c293064 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:47:28 +0300 Subject: [PATCH 690/950] Set query_id for mutations/merges This will allow to distinguish allocations in trace_log. --- src/Common/ThreadStatus.h | 8 ++++++++ src/Storages/MergeTree/MergeList.cpp | 7 ++++++- src/Storages/MergeTree/MergeList.h | 4 +++- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 5 ++++- src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp | 5 ++++- src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp | 5 ++++- 6 files changed, 29 insertions(+), 5 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 16a47a21184..d22f6e1574b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -41,6 +41,7 @@ class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; +class MemoryTrackerThreadSwitcher; using InternalTextLogsQueuePtr = std::shared_ptr; using InternalTextLogsQueueWeakPtr = std::weak_ptr; @@ -166,6 +167,13 @@ protected: /// It is used to avoid enabling the query profiler when you have multiple ThreadStatus in the same thread bool query_profiled_enabled = true; + /// Requires access to query_id. + friend class MemoryTrackerThreadSwitcher; + void setQueryId(const String & query_id_) + { + query_id = query_id_; + } + public: ThreadStatus(); ~ThreadStatus(); diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 8e85bcb986f..7ef61bbc382 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -10,7 +10,7 @@ namespace DB { -MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MemoryTracker * memory_tracker_ptr, UInt64 untracked_memory_limit) +MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MemoryTracker * memory_tracker_ptr, UInt64 untracked_memory_limit, const std::string & query_id) { // Each merge is executed into separate background processing pool thread background_thread_memory_tracker = CurrentThread::getMemoryTracker(); @@ -39,6 +39,9 @@ MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MemoryTracker * memory_ /// (NOTE: consider moving such code to ThreadFromGlobalPool and related places) prev_untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = 0; + + prev_query_id = current_thread->getQueryId().toString(); + current_thread->setQueryId(query_id); } @@ -51,6 +54,7 @@ MemoryTrackerThreadSwitcher::~MemoryTrackerThreadSwitcher() current_thread->untracked_memory_limit = prev_untracked_memory_limit; current_thread->untracked_memory = prev_untracked_memory; + current_thread->setQueryId(prev_query_id); } MergeListElement::MergeListElement( @@ -66,6 +70,7 @@ MergeListElement::MergeListElement( , result_part_info{future_part->part_info} , num_parts{future_part->parts.size()} , max_untracked_memory(max_untracked_memory_) + , query_id(table_id.getShortName() + "::" + result_part_name) , thread_id{getThreadId()} , merge_type{future_part->merge_type} , merge_algorithm{MergeAlgorithm::Undecided} diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 0389736a7f9..236837e3768 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -62,13 +62,14 @@ using FutureMergedMutatedPartPtr = std::shared_ptr; class MemoryTrackerThreadSwitcher : boost::noncopyable { public: - explicit MemoryTrackerThreadSwitcher(MemoryTracker * memory_tracker_ptr, UInt64 untracked_memory_limit); + explicit MemoryTrackerThreadSwitcher(MemoryTracker * memory_tracker_ptr, UInt64 untracked_memory_limit, const std::string & query_id); ~MemoryTrackerThreadSwitcher(); private: MemoryTracker * background_thread_memory_tracker; MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; UInt64 prev_untracked_memory_limit; UInt64 prev_untracked_memory; + String prev_query_id; }; using MemoryTrackerThreadSwitcherPtr = std::unique_ptr; @@ -107,6 +108,7 @@ struct MergeListElement : boost::noncopyable MemoryTracker memory_tracker{VariableContext::Process}; UInt64 max_untracked_memory; + std::string query_id; UInt64 thread_id; MergeType merge_type; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index af54bf113d6..5d5b22bdcd9 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -30,7 +30,10 @@ bool MergePlainMergeTreeTask::executeStep() /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; if (merge_list_entry) - switcher = std::make_unique(&(*merge_list_entry)->memory_tracker, (*merge_list_entry)->max_untracked_memory); + switcher = std::make_unique( + &(*merge_list_entry)->memory_tracker, + (*merge_list_entry)->max_untracked_memory, + (*merge_list_entry)->query_id); switch (state) { diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 39a057d68ca..a8faf8e7899 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -62,7 +62,10 @@ bool MutatePlainMergeTreeTask::executeStep() /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; if (merge_list_entry) - switcher = std::make_unique(&(*merge_list_entry)->memory_tracker, (*merge_list_entry)->max_untracked_memory); + switcher = std::make_unique( + &(*merge_list_entry)->memory_tracker, + (*merge_list_entry)->max_untracked_memory, + (*merge_list_entry)->query_id); switch (state) { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index 32071566bbe..a60fd42eec6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -122,7 +122,10 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() { MemoryTrackerThreadSwitcherPtr switcher; if (merge_mutate_entry) - switcher = std::make_unique(&(*merge_mutate_entry)->memory_tracker, (*merge_mutate_entry)->max_untracked_memory); + switcher = std::make_unique( + &(*merge_mutate_entry)->memory_tracker, + (*merge_mutate_entry)->max_untracked_memory, + (*merge_mutate_entry)->query_id); auto remove_processed_entry = [&] () -> bool { From 2a7a1d8df59da9e02ca27e5193a477b1209d697d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:47:28 +0300 Subject: [PATCH 691/950] Avoid losing any allocations context from merges/mutations --- src/Storages/MergeTree/MergeList.cpp | 14 +++++++++----- src/Storages/MergeTree/MergeList.h | 13 ++++++++++--- src/Storages/MergeTree/MergePlainMergeTreeTask.cpp | 5 +---- .../MergeTree/MutatePlainMergeTreeTask.cpp | 5 +---- .../MergeTree/ReplicatedMergeMutateTaskBase.cpp | 5 +---- 5 files changed, 22 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeList.cpp b/src/Storages/MergeTree/MergeList.cpp index 7ef61bbc382..366834b8f09 100644 --- a/src/Storages/MergeTree/MergeList.cpp +++ b/src/Storages/MergeTree/MergeList.cpp @@ -10,7 +10,8 @@ namespace DB { -MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MemoryTracker * memory_tracker_ptr, UInt64 untracked_memory_limit, const std::string & query_id) +MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MergeListEntry & merge_list_entry_) + : merge_list_entry(merge_list_entry_) { // Each merge is executed into separate background processing pool thread background_thread_memory_tracker = CurrentThread::getMemoryTracker(); @@ -29,19 +30,19 @@ MemoryTrackerThreadSwitcher::MemoryTrackerThreadSwitcher(MemoryTracker * memory_ } background_thread_memory_tracker_prev_parent = background_thread_memory_tracker->getParent(); - background_thread_memory_tracker->setParent(memory_tracker_ptr); + background_thread_memory_tracker->setParent(&merge_list_entry->memory_tracker); } prev_untracked_memory_limit = current_thread->untracked_memory_limit; - current_thread->untracked_memory_limit = untracked_memory_limit; + current_thread->untracked_memory_limit = merge_list_entry->max_untracked_memory; /// Avoid accounting memory from another mutation/merge /// (NOTE: consider moving such code to ThreadFromGlobalPool and related places) prev_untracked_memory = current_thread->untracked_memory; - current_thread->untracked_memory = 0; + current_thread->untracked_memory = merge_list_entry->untracked_memory; prev_query_id = current_thread->getQueryId().toString(); - current_thread->setQueryId(query_id); + current_thread->setQueryId(merge_list_entry->query_id); } @@ -53,7 +54,10 @@ MemoryTrackerThreadSwitcher::~MemoryTrackerThreadSwitcher() background_thread_memory_tracker->setParent(background_thread_memory_tracker_prev_parent); current_thread->untracked_memory_limit = prev_untracked_memory_limit; + + merge_list_entry->untracked_memory = current_thread->untracked_memory; current_thread->untracked_memory = prev_untracked_memory; + current_thread->setQueryId(prev_query_id); } diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index 236837e3768..4139e9599aa 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -55,6 +55,10 @@ struct MergeInfo struct FutureMergedMutatedPart; using FutureMergedMutatedPartPtr = std::shared_ptr; +struct MergeListElement; +using MergeListEntry = BackgroundProcessListEntry; + + /** * Since merge is executed with multiple threads, this class * switches the parent MemoryTracker to account all the memory used. @@ -62,9 +66,10 @@ using FutureMergedMutatedPartPtr = std::shared_ptr; class MemoryTrackerThreadSwitcher : boost::noncopyable { public: - explicit MemoryTrackerThreadSwitcher(MemoryTracker * memory_tracker_ptr, UInt64 untracked_memory_limit, const std::string & query_id); + explicit MemoryTrackerThreadSwitcher(MergeListEntry & merge_list_entry_); ~MemoryTrackerThreadSwitcher(); private: + MergeListEntry & merge_list_entry; MemoryTracker * background_thread_memory_tracker; MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr; UInt64 prev_untracked_memory_limit; @@ -107,7 +112,11 @@ struct MergeListElement : boost::noncopyable std::atomic columns_written{}; MemoryTracker memory_tracker{VariableContext::Process}; + /// Used to adjust ThreadStatus::untracked_memory_limit UInt64 max_untracked_memory; + /// Used to avoid losing any allocation context + UInt64 untracked_memory = 0; + /// Used for identifying mutations/merges in trace_log std::string query_id; UInt64 thread_id; @@ -129,8 +138,6 @@ struct MergeListElement : boost::noncopyable ~MergeListElement(); }; -using MergeListEntry = BackgroundProcessListEntry; - /** Maintains a list of currently running merges. * For implementation of system.merges table. */ diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index 5d5b22bdcd9..1cbf928047c 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -30,10 +30,7 @@ bool MergePlainMergeTreeTask::executeStep() /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; if (merge_list_entry) - switcher = std::make_unique( - &(*merge_list_entry)->memory_tracker, - (*merge_list_entry)->max_untracked_memory, - (*merge_list_entry)->query_id); + switcher = std::make_unique(*merge_list_entry); switch (state) { diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index a8faf8e7899..3359693fa22 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -62,10 +62,7 @@ bool MutatePlainMergeTreeTask::executeStep() /// Make out memory tracker a parent of current thread memory tracker MemoryTrackerThreadSwitcherPtr switcher; if (merge_list_entry) - switcher = std::make_unique( - &(*merge_list_entry)->memory_tracker, - (*merge_list_entry)->max_untracked_memory, - (*merge_list_entry)->query_id); + switcher = std::make_unique(*merge_list_entry); switch (state) { diff --git a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp index a60fd42eec6..5fe7de70a20 100644 --- a/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeMutateTaskBase.cpp @@ -122,10 +122,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl() { MemoryTrackerThreadSwitcherPtr switcher; if (merge_mutate_entry) - switcher = std::make_unique( - &(*merge_mutate_entry)->memory_tracker, - (*merge_mutate_entry)->max_untracked_memory, - (*merge_mutate_entry)->query_id); + switcher = std::make_unique(*merge_mutate_entry); auto remove_processed_entry = [&] () -> bool { From 8db2c844d37dfd6756dc21cb5addf4f3004553a0 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 02:07:52 +0300 Subject: [PATCH 692/950] Fixed test --- ...t_add_database.sql => 02097_default_dict_get_add_database.sql} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{02098_default_dict_get_add_database.sql => 02097_default_dict_get_add_database.sql} (100%) diff --git a/tests/queries/0_stateless/02098_default_dict_get_add_database.sql b/tests/queries/0_stateless/02097_default_dict_get_add_database.sql similarity index 100% rename from tests/queries/0_stateless/02098_default_dict_get_add_database.sql rename to tests/queries/0_stateless/02097_default_dict_get_add_database.sql From 07adc847db1ce17bcb96435b7347eb8ce4cd938d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 15 Oct 2021 10:05:39 +0300 Subject: [PATCH 693/950] Fix clang-tidy warnings in FunctionsJSON code --- src/Functions/FunctionsJSON.cpp | 32 +++++++++++++------------------- src/Functions/FunctionsJSON.h | 17 +++++++++++++++++ 2 files changed, 30 insertions(+), 19 deletions(-) create mode 100644 src/Functions/FunctionsJSON.h diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 12b4ca76ec2..6889a0d44b9 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1,5 +1,3 @@ -#include - #include #include @@ -40,6 +38,7 @@ #include #include #include +#include #include @@ -157,12 +156,6 @@ private: BOOST_TTI_HAS_MEMBER_FUNCTION(reserve) BOOST_TTI_HAS_MEMBER_FUNCTION(prepare) - template - struct has_index_operator : std::false_type {}; - - template - struct has_index_operator()[0])>> : std::true_type {}; - /// Represents a move of a JSON iterator described by a single argument passed to a JSON function. /// For example, the call JSONExtractInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 1) /// contains two moves: {MoveType::ConstKey, "b"} and {MoveType::ConstIndex, 1}. @@ -177,7 +170,7 @@ private: struct Move { - Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} + explicit Move(MoveType type_, size_t index_ = 0) : type(type_), index(index_) {} Move(MoveType type_, const String & key_) : type(type_), key(key_) {} MoveType type; size_t index = 0; @@ -286,7 +279,7 @@ private: return true; } - if constexpr (has_index_operator::value) + if constexpr (FunctionJSONHelpersDetails::has_index_operator::value) { if (element.isObject()) { @@ -838,7 +831,7 @@ struct JSONExtractTree class DecimalNode : public Node { public: - DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} + explicit DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { if (!element.isDouble()) @@ -887,7 +880,7 @@ struct JSONExtractTree class EnumNode : public Node { public: - EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) + explicit EnumNode(const std::vector> & name_value_pairs_) : name_value_pairs(name_value_pairs_) { for (const auto & name_value_pair : name_value_pairs) { @@ -939,7 +932,7 @@ struct JSONExtractTree class NullableNode : public Node { public: - NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} + explicit NullableNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { @@ -957,7 +950,7 @@ struct JSONExtractTree class ArrayNode : public Node { public: - ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} + explicit ArrayNode(std::unique_ptr nested_) : nested(std::move(nested_)) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { @@ -1125,6 +1118,7 @@ struct JSONExtractTree const auto & tuple = static_cast(*type); const auto & tuple_elements = tuple.getElements(); std::vector> elements; + elements.reserve(tuple_elements.size()); for (const auto & tuple_element : tuple_elements) elements.emplace_back(build(function_name, tuple_element)); return std::make_unique(std::move(elements), tuple.haveExplicitNames() ? tuple.getElementNames() : Strings{}); @@ -1148,7 +1142,7 @@ public: throw Exception{"Function " + String(function_name) + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const auto & col = arguments.back(); - auto col_type_const = typeid_cast(col.column.get()); + const auto * col_type_const = typeid_cast(col.column.get()); if (!col_type_const || !isString(col.type)) throw Exception{"The last argument of function " + String(function_name) + " should be a constant string specifying the return data type, illegal value: " + col.name, @@ -1186,7 +1180,7 @@ public: throw Exception{"Function " + String(function_name) + " requires at least two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; const auto & col = arguments.back(); - auto col_type_const = typeid_cast(col.column.get()); + const auto * col_type_const = typeid_cast(col.column.get()); if (!col_type_const || !isString(col.type)) throw Exception{"The last argument of function " + String(function_name) + " should be a constant string specifying the values' data type, illegal value: " + col.name, @@ -1291,7 +1285,7 @@ private: } if (element.isString()) { - writeJSONString(element.getString(), buf, format_settings()); + writeJSONString(element.getString(), buf, formatSettings()); return; } if (element.isArray()) @@ -1315,7 +1309,7 @@ private: { if (std::exchange(need_comma, true)) writeChar(',', buf); - writeJSONString(key, buf, format_settings()); + writeJSONString(key, buf, formatSettings()); writeChar(':', buf); traverse(value, buf); } @@ -1329,7 +1323,7 @@ private: } } - static const FormatSettings & format_settings() + static const FormatSettings & formatSettings() { static const FormatSettings the_instance = [] { diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h new file mode 100644 index 00000000000..4ef43eb637f --- /dev/null +++ b/src/Functions/FunctionsJSON.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +namespace DB +{ + +namespace FunctionJSONHelpersDetails +{ + template + struct has_index_operator : std::false_type {}; + + template + struct has_index_operator()[0])>> : std::true_type {}; +} + +} From 4ab6f7d771a453eb4f098d5a666b8170eb565f76 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Oct 2021 10:39:31 +0300 Subject: [PATCH 694/950] Finally fix test --- tests/queries/1_stateful/00167_read_bytes_from_fs.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql index ee3e6b94537..ac20e60b177 100644 --- a/tests/queries/1_stateful/00167_read_bytes_from_fs.sql +++ b/tests/queries/1_stateful/00167_read_bytes_from_fs.sql @@ -1,7 +1,7 @@ SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40; -SYSTEM FLUSH LOGS; - -- We had a bug which lead to additional compressed data read. test.hits compressed size is about 1.2Gb, but we read more then 3Gb. -- Small additional reads still possible, so we compare with about 1.5Gb. -SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40' and current_database = currentDatabase() and type = 'QueryFinish'; +SYSTEM FLUSH LOGS; + +SELECT ProfileEvents['ReadBufferFromFileDescriptorReadBytes'] < 1500000000 from system.query_log where query = 'SELECT sum(cityHash64(*)) FROM test.hits SETTINGS max_threads=40;' and current_database = currentDatabase() and type = 'QueryFinish'; From 766e08567c4e1e28c9fae14564d03ee1c6633a8b Mon Sep 17 00:00:00 2001 From: lhuang09287750 Date: Fri, 15 Oct 2021 08:09:37 +0000 Subject: [PATCH 695/950] complete date32 comparison with date/datetime/string and give a test file --- src/DataTypes/getLeastSupertype.cpp | 19 +++++++++++++++---- src/Functions/FunctionsComparison.h | 7 ++++--- src/Interpreters/convertFieldToType.cpp | 6 ++++++ .../02098_date32_comparison.reference | 6 ++++++ .../0_stateless/02098_date32_comparison.sql | 6 ++++++ 5 files changed, 37 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02098_date32_comparison.reference create mode 100644 tests/queries/0_stateless/02098_date32_comparison.sql diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index a950d18b50a..b77fcdcdfca 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -277,17 +277,18 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For Date and DateTime/DateTime64, the common type is DateTime/DateTime64. No other types are compatible. { UInt32 have_date = type_ids.count(TypeIndex::Date); + UInt32 have_date32 = type_ids.count(TypeIndex::Date32); UInt32 have_datetime = type_ids.count(TypeIndex::DateTime); UInt32 have_datetime64 = type_ids.count(TypeIndex::DateTime64); - if (have_date || have_datetime || have_datetime64) + if (have_date || have_date32 || have_datetime || have_datetime64) { - bool all_date_or_datetime = type_ids.size() == (have_date + have_datetime + have_datetime64); + bool all_date_or_datetime = type_ids.size() == (have_date + have_date32 + have_datetime + have_datetime64); if (!all_date_or_datetime) - throw Exception(getExceptionMessagePrefix(types) + " because some of them are Date/DateTime/DateTime64 and some of them are not", + throw Exception(getExceptionMessagePrefix(types) + " because some of them are Date/Date32/DateTime/DateTime64 and some of them are not", ErrorCodes::NO_COMMON_TYPE); - if (have_datetime64 == 0) + if (have_datetime64 == 0 && have_date32 == 0) { for (const auto & type : types) { @@ -298,6 +299,16 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return std::make_shared(); } + /// For Date and Date32, the common type is Date32 + if (have_datetime == 0 && have_datetime64 == 0) + { + for (const auto & type : types) + { + if (isDate32(type)) + return type; + } + } + UInt8 max_scale = 0; size_t max_scale_date_time_index = 0; diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 31356deb3fe..9b94ac589a3 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1088,7 +1088,7 @@ public: if (!((both_represented_by_number && !has_date) /// Do not allow to compare date and number. || (left.isStringOrFixedString() || right.isStringOrFixedString()) /// Everything can be compared with string by conversion. /// You can compare the date, datetime, or datatime64 and an enumeration with a constant string. - || ((left.isDate() || left.isDateTime() || left.isDateTime64()) && (right.isDate() || right.isDateTime() || right.isDateTime64()) && left.idx == right.idx) /// only date vs date, or datetime vs datetime + || ((left.isDate() || left.isDate32() || left.isDateTime() || left.isDateTime64()) && (right.isDate() || right.isDate32() || right.isDateTime() || right.isDateTime64()) && left.idx == right.idx) /// only date vs date, or datetime vs datetime || (left.isUUID() && right.isUUID()) || (left.isEnum() && right.isEnum() && arguments[0]->getName() == arguments[1]->getName()) /// only equivalent enum type values can be compared against || (left_tuple && right_tuple && left_tuple->getElements().size() == right_tuple->getElements().size()) @@ -1178,8 +1178,8 @@ public: const bool left_is_string = isStringOrFixedString(which_left); const bool right_is_string = isStringOrFixedString(which_right); - bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDateTime() || which_left.isDateTime64()) - && (which_right.isDate() || which_right.isDateTime() || which_right.isDateTime64()); + bool date_and_datetime = (which_left.idx != which_right.idx) && (which_left.isDate() || which_left.isDate32() || which_left.isDateTime() || which_left.isDateTime64()) + && (which_right.isDate() || which_right.isDate32() || which_right.isDateTime() || which_right.isDateTime64()); ColumnPtr res; if (left_is_num && right_is_num && !date_and_datetime) @@ -1247,6 +1247,7 @@ public: ColumnPtr c0_converted = castColumn(col_with_type_and_name_left, common_type); ColumnPtr c1_converted = castColumn(col_with_type_and_name_right, common_type); if (!((res = executeNumLeftType(c0_converted.get(), c1_converted.get())) + || (res = executeNumLeftType(c0_converted.get(), c1_converted.get())) || (res = executeNumLeftType(c0_converted.get(), c1_converted.get())))) throw Exception("Date related common types can only be UInt32 or UInt64", ErrorCodes::LOGICAL_ERROR); return res; diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 57bb4aa4905..ef86f45b759 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -203,6 +203,12 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return src; } + if (which_type.isDate32() && src.getType() == Field::Types::Int64) + { + /// We don't need any conversion Int64 is under type of Date32 + return src; + } + if (which_type.isDateTime64() && src.getType() == Field::Types::Decimal64) { /// Already in needed type. diff --git a/tests/queries/0_stateless/02098_date32_comparison.reference b/tests/queries/0_stateless/02098_date32_comparison.reference new file mode 100644 index 00000000000..a9e2f17562a --- /dev/null +++ b/tests/queries/0_stateless/02098_date32_comparison.reference @@ -0,0 +1,6 @@ +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02098_date32_comparison.sql b/tests/queries/0_stateless/02098_date32_comparison.sql new file mode 100644 index 00000000000..d0a632977c4 --- /dev/null +++ b/tests/queries/0_stateless/02098_date32_comparison.sql @@ -0,0 +1,6 @@ +select toDate32('1990-01-01') = toDate('1990-01-01'); +select toDate('1991-01-02') > toDate32('1990-01-01'); +select toDate32('1990-01-01') = toDateTime('1990-01-01'); +select toDateTime('1991-01-02') > toDate32('1990-01-01'); +select toDate32('1990-01-01') = '1990-01-01'; +select '1991-01-02' > toDate32('1990-01-01'); \ No newline at end of file From 03a7f24fa3961f699732ecd1e81455f5ce370689 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 11:14:15 +0300 Subject: [PATCH 696/950] Fix build. --- programs/odbc-bridge/MainHandler.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 6a2e967d179..f3e28891344 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include From 0d635cf34ea9b785642ed8d84beab02eac0b8ac3 Mon Sep 17 00:00:00 2001 From: Tentoshka <45504699+Tentoshka@users.noreply.github.com> Date: Fri, 15 Oct 2021 15:55:03 +0700 Subject: [PATCH 697/950] Translation external-dicts-dict-sources.md Translation of untranslated texts --- .../external-dictionaries/external-dicts-dict-sources.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index d616960ce36..828d69cf26b 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -251,7 +251,7 @@ ClickHouse получает от ODBC-драйвера информацию о ### Выявленная уязвимость в функционировании ODBC словарей {#vyiavlennaia-uiazvimost-v-funktsionirovanii-odbc-slovarei} -!!! attention "Attention" +!!! attention "Внимание" При соединении с базой данных через ODBC можно заменить параметр соединения `Servername`. В этом случае, значения `USERNAME` и `PASSWORD` из `odbc.ini` отправляются на удаленный сервер и могут быть скомпрометированы. **Пример небезопасного использования** @@ -764,7 +764,7 @@ Setting fields: - `port` – Порт для соединения с PostgreSQL. Вы можете указать его для всех реплик или задать индивидуально для каждой релпики (внутри ``). - `user` – Имя пользователя для соединения с PostgreSQL. Вы можете указать его для всех реплик или задать индивидуально для каждой релпики (внутри ``). - `password` – Пароль для пользователя PostgreSQL. -- `replica` – Section of replica configurations. There can be multiple sections. +- `replica` – Раздел конфигурации реплик. Может быть несколько. - `replica/host` – хост PostgreSQL. - `replica/port` – порт PostgreSQL . - `replica/priority` – Приоритет реплики. Во время попытки соединения, ClickHouse будет перебирать реплики в порядке приоритет. Меньшее значение означает более высокий приоритет. From 49e9ea40ace0f9e17b08d6a2deb070e20aee94ee Mon Sep 17 00:00:00 2001 From: Pavel Date: Fri, 15 Oct 2021 14:02:57 +0500 Subject: [PATCH 698/950] Update string-functions.md --- docs/ru/sql-reference/functions/string-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index cbda5188881..95e265f0b54 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -34,15 +34,15 @@ empty(x) Запрос: ```sql -SELECT notempty('text'); +SELECT empty('text'); ``` Результат: ```text -┌─empty('')─┐ -│ 1 │ -└───────────┘ +┌─empty('text')─┐ +│ 0 │ +└───────────────┘ ``` ## notEmpty {#notempty} From ad8a344b46985fe75671e032ccb78982d266a9f0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 13:11:57 +0300 Subject: [PATCH 699/950] Move TTL streams and algo --- src/CMakeLists.txt | 1 + .../TTL}/ITTLAlgorithm.cpp | 2 +- .../TTL}/ITTLAlgorithm.h | 0 .../TTL}/TTLAggregationAlgorithm.cpp | 2 +- .../TTL}/TTLAggregationAlgorithm.h | 2 +- .../TTL}/TTLColumnAlgorithm.cpp | 2 +- .../TTL}/TTLColumnAlgorithm.h | 2 +- .../TTL}/TTLDeleteAlgorithm.cpp | 2 +- .../TTL}/TTLDeleteAlgorithm.h | 2 +- .../TTL}/TTLUpdateInfoAlgorithm.cpp | 2 +- .../TTL}/TTLUpdateInfoAlgorithm.h | 2 +- .../Transforms/TTLCalcTransform.cpp} | 4 +-- .../Transforms/TTLCalcTransform.h} | 2 +- .../Transforms/TTLTransform.cpp} | 10 +++--- .../Transforms/TTLTransform.h} | 4 +-- src/Storages/MergeTree/MergeTask.cpp | 34 +++++++++---------- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +-- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 4 +-- 19 files changed, 42 insertions(+), 41 deletions(-) rename src/{DataStreams => Processors/TTL}/ITTLAlgorithm.cpp (98%) rename src/{DataStreams => Processors/TTL}/ITTLAlgorithm.h (100%) rename src/{DataStreams => Processors/TTL}/TTLAggregationAlgorithm.cpp (99%) rename src/{DataStreams => Processors/TTL}/TTLAggregationAlgorithm.h (96%) rename src/{DataStreams => Processors/TTL}/TTLColumnAlgorithm.cpp (98%) rename src/{DataStreams => Processors/TTL}/TTLColumnAlgorithm.h (95%) rename src/{DataStreams => Processors/TTL}/TTLDeleteAlgorithm.cpp (97%) rename src/{DataStreams => Processors/TTL}/TTLDeleteAlgorithm.h (92%) rename src/{DataStreams => Processors/TTL}/TTLUpdateInfoAlgorithm.cpp (97%) rename src/{DataStreams => Processors/TTL}/TTLUpdateInfoAlgorithm.h (94%) rename src/{DataStreams/TTLCalcInputStream.cpp => Processors/Transforms/TTLCalcTransform.cpp} (97%) rename src/{DataStreams/TTLCalcInputStream.h => Processors/Transforms/TTLCalcTransform.h} (96%) rename src/{DataStreams/TTLBlockInputStream.cpp => Processors/Transforms/TTLTransform.cpp} (95%) rename src/{DataStreams/TTLBlockInputStream.h => Processors/Transforms/TTLTransform.h} (92%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..08755542ed1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -214,6 +214,7 @@ add_object_library(clickhouse_processors_transforms Processors/Transforms) add_object_library(clickhouse_processors_sources Processors/Sources) add_object_library(clickhouse_processors_sinks Processors/Sinks) add_object_library(clickhouse_processors_merges Processors/Merges) +add_object_library(clickhouse_processors_ttl Processors/TTL) add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms) add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) add_object_library(clickhouse_processors_queryplan_optimizations Processors/QueryPlan/Optimizations) diff --git a/src/DataStreams/ITTLAlgorithm.cpp b/src/Processors/TTL/ITTLAlgorithm.cpp similarity index 98% rename from src/DataStreams/ITTLAlgorithm.cpp rename to src/Processors/TTL/ITTLAlgorithm.cpp index 7513e0c6ce0..489941950b5 100644 --- a/src/DataStreams/ITTLAlgorithm.cpp +++ b/src/Processors/TTL/ITTLAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/ITTLAlgorithm.h b/src/Processors/TTL/ITTLAlgorithm.h similarity index 100% rename from src/DataStreams/ITTLAlgorithm.h rename to src/Processors/TTL/ITTLAlgorithm.h diff --git a/src/DataStreams/TTLAggregationAlgorithm.cpp b/src/Processors/TTL/TTLAggregationAlgorithm.cpp similarity index 99% rename from src/DataStreams/TTLAggregationAlgorithm.cpp rename to src/Processors/TTL/TTLAggregationAlgorithm.cpp index d012464ea14..5581892f16f 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.cpp +++ b/src/Processors/TTL/TTLAggregationAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLAggregationAlgorithm.h b/src/Processors/TTL/TTLAggregationAlgorithm.h similarity index 96% rename from src/DataStreams/TTLAggregationAlgorithm.h rename to src/Processors/TTL/TTLAggregationAlgorithm.h index c2f40bab6b9..0e4bf092ed6 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.h +++ b/src/Processors/TTL/TTLAggregationAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/DataStreams/TTLColumnAlgorithm.cpp b/src/Processors/TTL/TTLColumnAlgorithm.cpp similarity index 98% rename from src/DataStreams/TTLColumnAlgorithm.cpp rename to src/Processors/TTL/TTLColumnAlgorithm.cpp index 71ad2a4e38f..7cef77c185c 100644 --- a/src/DataStreams/TTLColumnAlgorithm.cpp +++ b/src/Processors/TTL/TTLColumnAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLColumnAlgorithm.h b/src/Processors/TTL/TTLColumnAlgorithm.h similarity index 95% rename from src/DataStreams/TTLColumnAlgorithm.h rename to src/Processors/TTL/TTLColumnAlgorithm.h index ddf963eaee2..30de77dcc2a 100644 --- a/src/DataStreams/TTLColumnAlgorithm.h +++ b/src/Processors/TTL/TTLColumnAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/DataStreams/TTLDeleteAlgorithm.cpp b/src/Processors/TTL/TTLDeleteAlgorithm.cpp similarity index 97% rename from src/DataStreams/TTLDeleteAlgorithm.cpp rename to src/Processors/TTL/TTLDeleteAlgorithm.cpp index ea7a0b235ec..eec6b21f949 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.cpp +++ b/src/Processors/TTL/TTLDeleteAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLDeleteAlgorithm.h b/src/Processors/TTL/TTLDeleteAlgorithm.h similarity index 92% rename from src/DataStreams/TTLDeleteAlgorithm.h rename to src/Processors/TTL/TTLDeleteAlgorithm.h index 8ab3f8b63e8..292a29bfa27 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.h +++ b/src/Processors/TTL/TTLDeleteAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp similarity index 97% rename from src/DataStreams/TTLUpdateInfoAlgorithm.cpp rename to src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp index 6a983d052c1..eba364aa2b8 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h similarity index 94% rename from src/DataStreams/TTLUpdateInfoAlgorithm.h rename to src/Processors/TTL/TTLUpdateInfoAlgorithm.h index 551211fc47f..45eecbde3d0 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.h +++ b/src/Processors/TTL/TTLUpdateInfoAlgorithm.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/DataStreams/TTLCalcInputStream.cpp b/src/Processors/Transforms/TTLCalcTransform.cpp similarity index 97% rename from src/DataStreams/TTLCalcInputStream.cpp rename to src/Processors/Transforms/TTLCalcTransform.cpp index fe85e40c003..31fb61239ef 100644 --- a/src/DataStreams/TTLCalcInputStream.cpp +++ b/src/Processors/Transforms/TTLCalcTransform.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include namespace DB { diff --git a/src/DataStreams/TTLCalcInputStream.h b/src/Processors/Transforms/TTLCalcTransform.h similarity index 96% rename from src/DataStreams/TTLCalcInputStream.h rename to src/Processors/Transforms/TTLCalcTransform.h index b6318026b8c..14592c07155 100644 --- a/src/DataStreams/TTLCalcInputStream.h +++ b/src/Processors/Transforms/TTLCalcTransform.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/Processors/Transforms/TTLTransform.cpp similarity index 95% rename from src/DataStreams/TTLBlockInputStream.cpp rename to src/Processors/Transforms/TTLTransform.cpp index b476f689e60..a515a50fafb 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/Processors/Transforms/TTLTransform.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,10 +8,10 @@ #include #include -#include -#include -#include -#include +#include +#include +#include +#include namespace DB { diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/Processors/Transforms/TTLTransform.h similarity index 92% rename from src/DataStreams/TTLBlockInputStream.h rename to src/Processors/Transforms/TTLTransform.h index 50b28e81bdf..9207c68448b 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/Processors/Transforms/TTLTransform.h @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e17a3dcf544..7cf52058558 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1,26 +1,26 @@ -#include "Storages/MergeTree/MergeTask.h" +#include #include #include #include -#include "Common/ActionBlocker.h" +#include -#include "Storages/MergeTree/MergeTreeData.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" -#include "Storages/MergeTree/MergeTreeSequentialSource.h" -#include "Storages/MergeTree/FutureMergedMutatedPart.h" -#include "Processors/Transforms/ExpressionTransform.h" -#include "Processors/Transforms/MaterializingTransform.h" -#include "Processors/Merges/MergingSortedTransform.h" -#include "Processors/Merges/CollapsingSortedTransform.h" -#include "Processors/Merges/SummingSortedTransform.h" -#include "Processors/Merges/ReplacingSortedTransform.h" -#include "Processors/Merges/GraphiteRollupSortedTransform.h" -#include "Processors/Merges/AggregatingSortedTransform.h" -#include "Processors/Merges/VersionedCollapsingTransform.h" -#include "DataStreams/TTLBlockInputStream.h" -#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 1ee8423b0d1..5a199610a68 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -15,8 +15,8 @@ #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 50ab26ec470..812299828d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6ad29d01ca6..da79b917b5a 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include #include From 65aec857ec89897a98693ccbf0d65a8e311b6b16 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 13:12:59 +0300 Subject: [PATCH 700/950] FunctionsJSON updated --- src/Functions/FunctionsJSON.cpp | 9 ++++++++- src/Functions/FunctionsJSON.h | 17 ----------------- 2 files changed, 8 insertions(+), 18 deletions(-) delete mode 100644 src/Functions/FunctionsJSON.h diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 6889a0d44b9..4e2c42ea3cb 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -58,6 +59,12 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +template +struct HasIndexOperator : std::false_type {}; + +template +struct HasIndexOperator()[0])>> : std::true_type {}; + /// Functions to parse JSONs and extract values from it. /// The first argument of all these functions gets a JSON, @@ -279,7 +286,7 @@ private: return true; } - if constexpr (FunctionJSONHelpersDetails::has_index_operator::value) + if constexpr (HasIndexOperator::value) { if (element.isObject()) { diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h deleted file mode 100644 index 4ef43eb637f..00000000000 --- a/src/Functions/FunctionsJSON.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -namespace FunctionJSONHelpersDetails -{ - template - struct has_index_operator : std::false_type {}; - - template - struct has_index_operator()[0])>> : std::true_type {}; -} - -} From 41dc195b345c199e4fef4d62d57e9d1cab3cf69c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 13:15:14 +0300 Subject: [PATCH 701/950] Fix build. --- programs/odbc-bridge/MainHandler.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index f3e28891344..6362c7dfa9b 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include From df81d3f74a630c3d674eb51b9116d139419f8707 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 15 Oct 2021 14:52:24 +0300 Subject: [PATCH 702/950] Fix build in fast test --- src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp index b5e2b83c23b..58f88c5c7cf 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.cpp @@ -262,7 +262,7 @@ void registerOutputFormatCapnProto(FormatFactory & factory) namespace DB { class FormatFactory; -void registerOutputFormatProcessorsCapnProto(FormatFactory &) {} +void registerOutputFormatCapnProto(FormatFactory &) {} } #endif // USE_CAPNP From 5b6b8cf56433b0b8c3e0a4e645eb21577493576f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 15 Oct 2021 15:14:44 +0300 Subject: [PATCH 703/950] fix clang-tidy --- src/DataTypes/DataTypeTuple.cpp | 1 - src/DataTypes/Serializations/ISerialization.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 6bca7f2bac2..4e1a5a05d45 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -27,7 +27,6 @@ namespace ErrorCodes extern const int DUPLICATE_COLUMN; extern const int EMPTY_DATA_PASSED; extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH; } diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index a6d9185429c..5c0274b0e35 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -103,7 +103,7 @@ namespace using SubstreamIterator = ISerialization::SubstreamPath::const_iterator; -static String getNameForSubstreamPath( +String getNameForSubstreamPath( String stream_name, SubstreamIterator begin, SubstreamIterator end, From a744097fb65fbfa22809c5c640f4ff73f07fd2ba Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 15:34:13 +0300 Subject: [PATCH 704/950] Fixed build --- src/Functions/FunctionsJSON.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 4e2c42ea3cb..9558b856511 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -39,7 +39,6 @@ #include #include #include -#include #include From f69daace7b400735dec36e9190add9fd9e7651c9 Mon Sep 17 00:00:00 2001 From: Artur Filatenkov <613623@mail.ru> Date: Fri, 15 Oct 2021 16:01:28 +0300 Subject: [PATCH 705/950] attemp to fix build --- contrib/boringssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boringssl b/contrib/boringssl index ea9add0d76b..c1e01a441d6 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit ea9add0d76b0d2ff8616c5e9035389cd159996b7 +Subproject commit c1e01a441d6db234f4f12e63a7657d1f9e6db9c1 From e9bf496d42435c02a96518adf33e19dba4c4f004 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Oct 2021 14:35:50 +0000 Subject: [PATCH 706/950] Added an ability to generate data in build time --- CMakeLists.txt | 15 ++++++ .../libprotobuf-mutator-cmake/CMakeLists.txt | 7 +-- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 48 +++++++++++++++++-- .../codegen_fuzzer/codegen_select_fuzzer.cpp | 1 - src/Parsers/fuzzers/codegen_fuzzer/gen.py | 19 ++++---- 5 files changed, 70 insertions(+), 20 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f32841c15da..c2d3421b28e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -136,6 +136,21 @@ if (ENABLE_FUZZING) message (STATUS "Fuzzing instrumentation enabled") set (FUZZER "libfuzzer") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -nostdlib++") + set (ENABLE_CLICKHOUSE_ODBC_BRIDGE OFF) + set (ENABLE_LIBRARIES 0) + set (ENABLE_SSL 1) + set (USE_INTERNAL_SSL_LIBRARY 1) + set (USE_UNWIND ON) + set (ENABLE_EMBEDDED_COMPILER 0) + set (ENABLE_EXAMPLES 0) + set (ENABLE_UTILS 0) + set (ENABLE_THINLTO 0) + set (ENABLE_TCMALLOC 0) + set (ENABLE_JEMALLOC 0) + set (ENABLE_CHECK_HEAVY_BUILDS 1) + set (GLIBC_COMPATIBILITY OFF) + set (ENABLE_PROTOBUF ON) + set (USE_INTERNAL_PROTOBUF_LIBRARY ON) endif() # Global libraries diff --git a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt index 93eafc85b7d..978b1e732ba 100644 --- a/contrib/libprotobuf-mutator-cmake/CMakeLists.txt +++ b/contrib/libprotobuf-mutator-cmake/CMakeLists.txt @@ -1,5 +1,4 @@ set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator) -set(NO_FUZZING_FLAGS "-fno-sanitize=fuzzer -fsanitize-coverage=0") add_library(protobuf-mutator ${LIBRARY_DIR}/src/libfuzzer/libfuzzer_macro.cc @@ -10,8 +9,6 @@ add_library(protobuf-mutator ${LIBRARY_DIR}/src/utf8_fix.cc) target_include_directories(protobuf-mutator BEFORE PRIVATE "${LIBRARY_DIR}") -# target_include_directories(protobuf-mutator BEFORE PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") +target_include_directories(protobuf-mutator BEFORE PRIVATE "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") -target_link_libraries(protobuf-mutator ${PROTOBUF_LIBRARIES}) -set_target_properties(protobuf-mutator PROPERTIES - COMPILE_FLAGS "${NO_FUZZING_FLAGS}") +target_link_libraries(protobuf-mutator ${Protobuf_LIBRARY}) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 370396a5e8e..3c274bd4786 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -1,7 +1,47 @@ find_package(Protobuf REQUIRED) -protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS "out.proto") -set(FUZZER_SRCS codegen_select_fuzzer.cpp out.cpp ${PROTO_SRCS} ${PROTO_HDRS}) +set (CURRENT_DIR_IN_SOURCES "${ClickHouse_SOURCE_DIR}/src/Parsers/fuzzers/codegen_fuzzer") +set (CURRENT_DIR_IN_BINARY "${ClickHouse_BINARY_DIR}/src/Parsers/fuzzers/codegen_fuzzer") + +# Copy scripts and template file to build directory to generate .proto and .cpp file from them +configure_file( + "${CURRENT_DIR_IN_SOURCES}/gen.py" + "${CURRENT_DIR_IN_BINARY}/gen.py" + COPYONLY) +configure_file( + "${CURRENT_DIR_IN_SOURCES}/update.sh" + "${CURRENT_DIR_IN_BINARY}/update.sh" + COPYONLY) + +# Delete this and uncomment the next block if you want to generage clickhouse.g automatically +configure_file( + "${CURRENT_DIR_IN_SOURCES}/clickhouse.g" + "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + COPYONLY) + +# # Uncomment to generate clickhouse.g automatically +# configure_file( +# "${CURRENT_DIR_IN_SOURCES}/clickhouse-template.g" +# "${CURRENT_DIR_IN_BINARY}/clickhouse-template.g" +# COPYONLY) + +# # Note that it depends on all.dict file! +# add_custom_command( +# OUTPUT +# "${CURRENT_DIR_IN_BINARY}/clickhouse.g" +# COMMAND ./update.sh "${ClickHouse_SOURCE_DIR}/tests/fuzz/all.dict" +# ) + +add_custom_command( + OUTPUT + "${CURRENT_DIR_IN_BINARY}/out.cpp" + "${CURRENT_DIR_IN_BINARY}/out.proto" + COMMAND python3 gen.py clickhouse.g out.cpp out.proto + # DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" +) + +PROTOBUF_GENERATE_CPP(PROTO_SRCS PROTO_HDRS "${CURRENT_DIR_IN_BINARY}/out.proto") +set(FUZZER_SRCS codegen_select_fuzzer.cpp "${CURRENT_DIR_IN_BINARY}/out.cpp" ${PROTO_SRCS} ${PROTO_HDRS}) set(CMAKE_INCLUDE_CURRENT_DIR TRUE) @@ -9,7 +49,7 @@ add_executable(codegen_select_fuzzer ${FUZZER_SRCS}) set_source_files_properties("${PROTO_SRCS}" "out.cpp" PROPERTIES COMPILE_FLAGS "-Wno-reserved-identifier") -target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIRS}") +target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${Protobuf_INCLUDE_DIR}" "${CMAKE_CURRENT_BINARY_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}") target_include_directories(codegen_select_fuzzer BEFORE PRIVATE "${LibProtobufMutator_SOURCE_DIR}/src") -target_link_libraries(codegen_select_fuzzer PRIVATE clickhouse_parsers protobuf-mutator ${Protobuf_LIBRARIES} ${LIB_FUZZING_ENGINE}) +target_link_libraries(codegen_select_fuzzer PRIVATE protobuf-mutator dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index a68fac4f585..418a5014657 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -11,7 +11,6 @@ #include "out.pb.h" - void GenerateSentence(const Sentence&, std::string &, int); diff --git a/src/Parsers/fuzzers/codegen_fuzzer/gen.py b/src/Parsers/fuzzers/codegen_fuzzer/gen.py index e96bc6ae9f6..b594d1026e3 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/gen.py +++ b/src/Parsers/fuzzers/codegen_fuzzer/gen.py @@ -58,14 +58,14 @@ class Parser: self.var_id = -1 self.cur_tok = None self.includes = [] - + self.proto = '' self.cpp = '' def parse_file(self, filename): with open(filename) as f: self.text = f.read() - + while self.parse_statement() is not None: pass @@ -90,12 +90,12 @@ class Parser: def parse_var_value(self): i = self.text.find(' ') - + id_, self.text = self.text[1:i], self.text[i+1:] self.var_id = int(id_) self.cur_tok = TOKEN_VAR return TOKEN_VAR - + def parse_txt_value(self): if self.text[0] != '"': raise Exception("parse_txt_value: expected quote at the start") @@ -116,7 +116,7 @@ class Parser: else: c, self.text = self.text[0], self.text[1:] self.t += c - + self.text = self.text[1:] self.cur_tok = TOKEN_TEXT return TOKEN_TEXT @@ -137,7 +137,7 @@ class Parser: index = self.text.find('\n') self.text = self.text[index:] - + def parse_statement(self): if self.skip_ws() is None: return None @@ -146,7 +146,7 @@ class Parser: if self.cur_tok == TOKEN_SLASH: self.skip_line() return TOKEN_SLASH - + chain = [] while self.cur_tok != TOKEN_SEMI: if self.cur_tok == TOKEN_TEXT: @@ -164,7 +164,7 @@ class Parser: def generate(self): self.proto = 'syntax = "proto3";\n\n' - self.cpp = '#include \n#include \n#include \n\n#include \n\n' + self.cpp = '#include \n#include \n#include \n\n#include \n\n' for incl_file in self.includes: self.cpp += f'#include "{incl_file}"\n' @@ -228,7 +228,7 @@ def main(args): p = Parser() p.add_include(include_filename) p.parse_file(input_file) - + cpp, proto = p.generate() proto = proto.replace('\t', ' ' * 4) @@ -246,4 +246,3 @@ if __name__ == '__main__': print(f"Usage {sys.argv[0]} ") sys.exit(1) main(sys.argv[1:]) - From 058081bcb2810082bf14eea36402860653169b44 Mon Sep 17 00:00:00 2001 From: Pavel Cheremushkin Date: Fri, 15 Oct 2021 18:18:02 +0300 Subject: [PATCH 707/950] fixed typo in gen.py; clickhouse.g applying Nikita's in build-time generation --- .../fuzzers/codegen_fuzzer/CMakeLists.txt | 23 +- .../fuzzers/codegen_fuzzer/clickhouse.g | 524 ------------------ src/Parsers/fuzzers/codegen_fuzzer/gen.py | 2 +- src/Parsers/fuzzers/codegen_fuzzer/update.sh | 4 +- 4 files changed, 11 insertions(+), 542 deletions(-) delete mode 100644 src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index 3c274bd4786..f87340b5024 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -13,24 +13,17 @@ configure_file( "${CURRENT_DIR_IN_BINARY}/update.sh" COPYONLY) -# Delete this and uncomment the next block if you want to generage clickhouse.g automatically configure_file( - "${CURRENT_DIR_IN_SOURCES}/clickhouse.g" - "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + "${CURRENT_DIR_IN_SOURCES}/clickhouse-template.g" + "${CURRENT_DIR_IN_BINARY}/clickhouse-template.g" COPYONLY) -# # Uncomment to generate clickhouse.g automatically -# configure_file( -# "${CURRENT_DIR_IN_SOURCES}/clickhouse-template.g" -# "${CURRENT_DIR_IN_BINARY}/clickhouse-template.g" -# COPYONLY) - -# # Note that it depends on all.dict file! -# add_custom_command( -# OUTPUT -# "${CURRENT_DIR_IN_BINARY}/clickhouse.g" -# COMMAND ./update.sh "${ClickHouse_SOURCE_DIR}/tests/fuzz/all.dict" -# ) +# Note that it depends on all.dict file! +add_custom_command( + OUTPUT + "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + COMMAND ./update.sh "${ClickHouse_SOURCE_DIR}/tests/fuzz/all.dict" +) add_custom_command( OUTPUT diff --git a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g b/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g deleted file mode 100644 index edd5acf513d..00000000000 --- a/src/Parsers/fuzzers/codegen_fuzzer/clickhouse.g +++ /dev/null @@ -1,524 +0,0 @@ -" "; -" "; -" "; -";"; - - -"(" $1 ")"; -"(" $1 ", " $2 ")"; -"(" $1 ", " $2 ", " $3 ")"; - -$1 ", " $2 ; -$1 ", " $2 ", " $3 ; -$1 ", " $2 ", " $3 ", " $4 ; -$1 ", " $2 ", " $3 ", " $4 ", " $5 ; - -"[" $1 ", " $2 "]"; -"[" $1 ", " $2 ", " $3 "]"; -"[" $1 ", " $2 ", " $3 ", " $4 "]"; -"[" $1 ", " $2 ", " $3 ", " $4 ", " $5 "]"; - -$0 "(" $1 ")"; -$0 "(" $1 ", " $2 ")"; -$0 "(" $1 ", " $2 ", " $3 ")"; - -$1 " as " $2 ; - - -// TODO: add more clickhouse specific stuff -"SELECT " $1 " FROM " $2 " WHERE " $3 ; -"SELECT " $1 " FROM " $2 " GROUP BY " $3 ; -"SELECT " $1 " FROM " $2 " SORT BY " $3 ; -"SELECT " $1 " FROM " $2 " LIMIT " $3 ; -"SELECT " $1 " FROM " $2 " JOIN " $3 ; -"SELECT " $1 " FROM " $2 " ARRAY JOIN " $3 ; -"SELECT " $1 " FROM " $2 " JOIN " $3 " ON " $4 ; -"SELECT " $1 " FROM " $2 " JOIN " $3 " USING " $5 ; -"SELECT " $1 " INTO OUTFILE " $2 ; - -"WITH " $1 " AS " $2 ; - -"{" $1 ":" $2 "}"; -"[" $1 "," $2 "]"; -"[]"; - - -" x "; -"x"; -" `x` "; -"`x`"; - -" \"value\" "; -"\"value\""; -" 0 "; -"0"; -"1"; -"2"; -"123123123123123123"; -"182374019873401982734091873420923123123123123123"; -"1e-1"; -"1.1"; -"\"\""; -" '../../../../../../../../../etc/passwd' "; - -"/"; -"="; -"=="; -"!="; -"<>"; -"<"; -"<="; -">"; -">="; -"<<"; -"|<<"; -"&"; -"|"; -"||"; -"<|"; -"|>"; -"+"; -"-"; -"~"; -"*"; -"/"; -"\\"; -"%"; -""; -"."; -","; -","; -","; -","; -","; -","; -"("; -")"; -"("; -")"; -"("; -")"; -"("; -")"; -"("; -")"; -"("; -")"; -"?"; -":"; -"@"; -"@@"; -"$"; -"\""; -"`"; -"{"; -"}"; -"^"; -"::"; -"->"; -"]"; -"["; - -" ADD "; -" ADD COLUMN "; -" ADD CONSTRAINT "; -" ADD INDEX "; -" AFTER "; -" AggregateFunction "; -" aggThrow "; -" ALL "; -" ALTER LIVE VIEW "; -" ALTER TABLE "; -" and "; -" ANTI "; -" any "; -" anyHeavy "; -" anyLast "; -" argMax "; -" argMin "; -" array "; -" Array "; -" arrayAll "; -" arrayAUC "; -" arrayCompact "; -" arrayConcat "; -" arrayCount "; -" arrayCumSum "; -" arrayCumSumNonNegative "; -" arrayDifference "; -" arrayDistinct "; -" arrayElement "; -" arrayEnumerate "; -" arrayEnumerateDense "; -" arrayEnumerateDenseRanked "; -" arrayEnumerateUniq "; -" arrayEnumerateUniqRanked "; -" arrayExists "; -" arrayFill "; -" arrayFilter "; -" arrayFirst "; -" arrayFirstIndex "; -" arrayFlatten "; -" arrayIntersect "; -" arrayJoin "; -" ARRAY JOIN "; -" arrayMap "; -" arrayPopBack "; -" arrayPopFront "; -" arrayPushBack "; -" arrayPushFront "; -" arrayReduce "; -" arrayReduceInRanges "; -" arrayResize "; -" arrayReverse "; -" arrayReverseFill "; -" arrayReverseSort "; -" arrayReverseSplit "; -" arraySlice "; -" arraySort "; -" arraySplit "; -" arraySum "; -" arrayUniq "; -" arrayWithConstant "; -" arrayZip "; -" AS "; -" ASC "; -" ASCENDING "; -" ASOF "; -" AST "; -" ATTACH "; -" ATTACH PART "; -" ATTACH PARTITION "; -" avg "; -" avgWeighted "; -" basename "; -" BETWEEN "; -" BOTH "; -" boundingRatio "; -" BY "; -" CAST "; -" categoricalInformationValue "; -" CHECK "; -" CHECK TABLE "; -" CLEAR COLUMN "; -" CLEAR INDEX "; -" COLLATE "; -" COLUMNS "; -" COMMENT COLUMN "; -" CONSTRAINT "; -" corr "; -" corrStable "; -" count "; -" countEqual "; -" covarPop "; -" covarPopStable "; -" covarSamp "; -" covarSampStable "; -" CREATE "; -" CROSS "; -" CUBE "; -" cutFragment "; -" cutQueryString "; -" cutQueryStringAndFragment "; -" cutToFirstSignificantSubdomain "; -" cutURLParameter "; -" cutWWW "; -" D "; -" DATABASE "; -" DATABASES "; -" Date "; -" DATE "; -" DATE_ADD "; -" DATEADD "; -" DATE_DIFF "; -" DATEDIFF "; -" DATE_SUB "; -" DATESUB "; -" DateTime "; -" DateTime64 "; -" DAY "; -" DD "; -" Decimal "; -" Decimal128 "; -" Decimal32 "; -" Decimal64 "; -" decodeURLComponent "; -" DEDUPLICATE "; -" DELETE "; -" DELETE WHERE "; -" DESC "; -" DESCENDING "; -" DESCRIBE "; -" DETACH "; -" DETACH PARTITION "; -" DICTIONARIES "; -" DICTIONARY "; -" DISTINCT "; -" domain "; -" domainWithoutWWW "; -" DROP "; -" DROP COLUMN "; -" DROP CONSTRAINT "; -" DROP DETACHED PART "; -" DROP DETACHED PARTITION "; -" DROP INDEX "; -" DROP PARTITION "; -" emptyArrayToSingle "; -" ENGINE "; -" entropy "; -" Enum "; -" Enum16 "; -" Enum8 "; -" EVENTS "; -" EXCHANGE TABLES "; -" EXISTS "; -" EXTRACT "; -" extractURLParameter "; -" extractURLParameterNames "; -" extractURLParameters "; -" FETCH PARTITION "; -" FETCH PART "; -" FINAL "; -" FIRST "; -" firstSignificantSubdomain "; -" FixedString "; -" Float32 "; -" Float64 "; -" FOR "; -" ForEach "; -" FORMAT "; -" fragment "; -" FREEZE "; -" FROM "; -" FULL "; -" FUNCTION "; -" __getScalar "; -" GLOBAL "; -" GRANULARITY "; -" groupArray "; -" groupArrayInsertAt "; -" groupArrayMovingAvg "; -" groupArrayMovingSum "; -" groupArraySample "; -" groupBitAnd "; -" groupBitmap "; -" groupBitmapAnd "; -" groupBitmapOr "; -" groupBitmapXor "; -" groupBitOr "; -" groupBitXor "; -" GROUP BY "; -" groupUniqArray "; -" has "; -" hasAll "; -" hasAny "; -" HAVING "; -" HH "; -" histogram "; -" HOUR "; -" ID "; -" if "; -" IF EXISTS "; -" IF NOT EXISTS "; -" IN "; -" INDEX "; -" indexOf "; -" INNER "; -" IN PARTITION "; -" INSERT INTO "; -" Int16 "; -" Int32 "; -" Int64 "; -" Int8 "; -" INTERVAL "; -" IntervalDay "; -" IntervalHour "; -" IntervalMinute "; -" IntervalMonth "; -" IntervalQuarter "; -" IntervalSecond "; -" IntervalWeek "; -" IntervalYear "; -" INTO OUTFILE "; -" JOIN "; -" kurtPop "; -" kurtSamp "; -" LAST "; -" LAYOUT "; -" LEADING "; -" LEFT "; -" LEFT ARRAY JOIN "; -" length "; -" LIFETIME "; -" LIKE "; -" LIMIT "; -" LIVE "; -" LOCAL "; -" LowCardinality "; -" LTRIM "; -" M "; -" MATERIALIZED "; -" MATERIALIZE INDEX "; -" MATERIALIZE TTL "; -" max "; -" maxIntersections "; -" maxIntersectionsPosition "; -" Merge "; -" MI "; -" min "; -" MINUTE "; -" MM "; -" MODIFY "; -" MODIFY COLUMN "; -" MODIFY ORDER BY "; -" MODIFY QUERY "; -" MODIFY SETTING "; -" MODIFY TTL "; -" MONTH "; -" MOVE PART "; -" MOVE PARTITION "; -" movingXXX "; -" N "; -" NAME "; -" Nested "; -" NO DELAY "; -" NONE "; -" not "; -" nothing "; -" Nothing "; -" Null "; -" Nullable "; -" NULLS "; -" OFFSET "; -" ON "; -" ONLY "; -" OPTIMIZE TABLE "; -" ORDER BY "; -" OR REPLACE "; -" OUTER "; -" PARTITION "; -" PARTITION BY "; -" path "; -" pathFull "; -" POPULATE "; -" PREWHERE "; -" PRIMARY KEY "; -" protocol "; -" Q "; -" QQ "; -" QUARTER "; -" queryString "; -" queryStringAndFragment "; -" range "; -" REFRESH "; -" RENAME COLUMN "; -" RENAME TABLE "; -" REPLACE PARTITION "; -" Resample "; -" RESUME "; -" retention "; -" RIGHT "; -" ROLLUP "; -" RTRIM "; -" S "; -" SAMPLE "; -" SAMPLE BY "; -" SECOND "; -" SELECT "; -" SEMI "; -" sequenceCount "; -" sequenceMatch "; -" SET "; -" SETTINGS "; -" SHOW "; -" SHOW PROCESSLIST "; -" simpleLinearRegression "; -" skewPop "; -" skewSamp "; -" SOURCE "; -" SQL_TSI_DAY "; -" SQL_TSI_HOUR "; -" SQL_TSI_MINUTE "; -" SQL_TSI_MONTH "; -" SQL_TSI_QUARTER "; -" SQL_TSI_SECOND "; -" SQL_TSI_WEEK "; -" SQL_TSI_YEAR "; -" SS "; -" State "; -" stddevPop "; -" stddevPopStable "; -" stddevSamp "; -" stddevSampStable "; -" STEP "; -" stochasticLinearRegression "; -" stochasticLogisticRegression "; -" String "; -" SUBSTRING "; -" sum "; -" sumKahan "; -" sumMap "; -" sumMapFiltered "; -" sumMapFilteredWithOverflow "; -" sumMapWithOverflow "; -" sumWithOverflow "; -" SUSPEND "; -" TABLE "; -" TABLES "; -" TEMPORARY "; -" TIMESTAMP "; -" TIMESTAMP_ADD "; -" TIMESTAMPADD "; -" TIMESTAMP_DIFF "; -" TIMESTAMPDIFF "; -" TIMESTAMP_SUB "; -" TIMESTAMPSUB "; -" TO "; -" TO DISK "; -" TOP "; -" topK "; -" topKWeighted "; -" topLevelDomain "; -" TO TABLE "; -" TOTALS "; -" TO VOLUME "; -" TRAILING "; -" TRIM "; -" TRUNCATE "; -" TTL "; -" Tuple "; -" TYPE "; -" UInt16 "; -" UInt32 "; -" UInt64 "; -" UInt8 "; -" uniq "; -" uniqCombined "; -" uniqCombined64 "; -" uniqExact "; -" uniqHLL12 "; -" uniqUpTo "; -" UPDATE "; -" URLHierarchy "; -" URLPathHierarchy "; -" USE "; -" USING "; -" UUID "; -" VALUES "; -" varPop "; -" varPopStable "; -" varSamp "; -" varSampStable "; -" VIEW "; -" WATCH "; -" WEEK "; -" WHERE "; -" windowFunnel "; -" WITH "; -" WITH FILL "; -" WITH TIES "; -" WK "; -" WW "; -" YEAR "; -" YY "; -" YYYY "; diff --git a/src/Parsers/fuzzers/codegen_fuzzer/gen.py b/src/Parsers/fuzzers/codegen_fuzzer/gen.py index b594d1026e3..95936247489 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/gen.py +++ b/src/Parsers/fuzzers/codegen_fuzzer/gen.py @@ -154,7 +154,7 @@ class Parser: elif self.cur_tok == TOKEN_VAR: chain.append(Var(self.var_id)) else: - self.fatal_parsing_error(f"unexpected token {self.tok}") + self.fatal_parsing_error(f"unexpected token {self.cur_tok}") self.get_next_token() if not chain: diff --git a/src/Parsers/fuzzers/codegen_fuzzer/update.sh b/src/Parsers/fuzzers/codegen_fuzzer/update.sh index 0982d6d0686..daee56dcea1 100755 --- a/src/Parsers/fuzzers/codegen_fuzzer/update.sh +++ b/src/Parsers/fuzzers/codegen_fuzzer/update.sh @@ -20,11 +20,11 @@ _main() { while read line; do [[ -z "$line" ]] && continue - echo $line | sed -e 's/"\(.*\)"/" \1 ";/g' + echo $line | sed -e '/^#/d' -e 's/"\(.*\)"/" \1 ";/g' done < $dict_filename >> clickhouse.g } _main "$@" -# Sample run: ./update.sh ../../../../tests/fuzz/ast.dict +# Sample run: ./update.sh ${CLICKHOUSE_SOURCE_DIR}/tests/fuzz/all.dict # then run `python ./gen.py clickhouse.g out.cpp out.proto` to generate new files with tokens. Rebuild fuzzer From f0452b34d502fabbc069dec438c8473a0fd0e7e3 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 15 Oct 2021 18:21:02 +0300 Subject: [PATCH 708/950] Update clickhouse-v21.10-released.md --- website/blog/en/2021/clickhouse-v21.10-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.10-released.md b/website/blog/en/2021/clickhouse-v21.10-released.md index e5d870147f4..5b6f6d8b9e3 100644 --- a/website/blog/en/2021/clickhouse-v21.10-released.md +++ b/website/blog/en/2021/clickhouse-v21.10-released.md @@ -2,7 +2,7 @@ title: 'ClickHouse v21.10 Released' image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-10/featured.jpg' date: '2021-10-14' -author: '[Alexey Milovidov](https://github.com/alexey-milovidov)' +author: '[Rich Raposa](https://github.com/rfraposa), [Alexey Milovidov](https://github.com/alexey-milovidov)' tags: ['company', 'community'] --- From 09b8d697bd2a27d07dd7b2621f3cdd788cb14d6f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:26:56 +0300 Subject: [PATCH 709/950] Flat Hashed dictionary fix bytes_allocated for nullable attributes --- src/Dictionaries/FlatDictionary.cpp | 5 +++++ src/Dictionaries/HashedDictionary.cpp | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index e14ee5d30d1..532c68d0453 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -403,6 +403,11 @@ void FlatDictionary::calculateBytesAllocated() }; callOnDictionaryAttributeType(attribute.type, type_call); + + bytes_allocated += sizeof(attribute.is_nullable_set); + + if (attribute.is_nullable_set.has_value()) + bytes_allocated = attribute.is_nullable_set->getBufferSizeInBytes(); } if (update_field_loaded_block) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1f3821096da..00d339865f8 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -626,6 +626,11 @@ void HashedDictionary::calculateBytesAllocated() if (attributes[i].string_arena) bytes_allocated += attributes[i].string_arena->size(); + + bytes_allocated += sizeof(attributes[i].is_nullable_set); + + if (attributes[i].is_nullable_set.has_value()) + bytes_allocated = attributes[i].is_nullable_set->getBufferSizeInBytes(); } bytes_allocated += complex_key_arena.size(); From be4fc79d32cfad558202d380141a449f7a543cbf Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 15 Oct 2021 18:29:27 +0300 Subject: [PATCH 710/950] Better handling exceptions, update tests --- programs/client/Client.cpp | 14 +++----- programs/local/LocalServer.cpp | 6 ++++ src/Client/ClientBase.cpp | 15 +++++--- .../01527_clickhouse_local_optimize.sh | 2 +- .../01528_clickhouse_local_prepare_parts.sh | 16 ++++----- .../0_stateless/01600_detach_permanently.sh | 2 +- ..._bad_options_in_client_and_local.reference | 12 +++++++ .../02096_bad_options_in_client_and_local.sh | 34 +++++++++++++++++++ ...known_option_in_clickhouse_local.reference | 2 -- ...2096_unknown_option_in_clickhouse_local.sh | 9 ----- 10 files changed, 78 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02096_bad_options_in_client_and_local.reference create mode 100755 tests/queries/0_stateless/02096_bad_options_in_client_and_local.sh delete mode 100644 tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference delete mode 100755 tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index a5e4bd45c7f..3c50acb1df6 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -25,9 +25,6 @@ #endif #include #include -#include -#include -#include #include #include #include "Common/MemoryTracker.h" @@ -35,13 +32,11 @@ #include #include #include -#include #include #include #include #include -#include #include #include @@ -53,16 +48,12 @@ #include #include #include -#include -#include -#include #include #include #include #include -#include #include "TestTags.h" #ifndef __clang__ @@ -1234,6 +1225,11 @@ int mainEntryClickHouseClient(int argc, char ** argv) std::cerr << DB::getExceptionMessage(e, false) << std::endl; return 1; } + catch (const boost::program_options::error & e) + { + std::cerr << "Bad arguments: " << e.what() << std::endl; + return DB::ErrorCodes::BAD_ARGUMENTS; + } catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2180729438d..9e67f04699a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -652,6 +652,7 @@ void LocalServer::addOptions(OptionsDescription & options_description) ("logger.level", po::value(), "Log level") ("no-system-tables", "do not attach system tables (better startup time)") + ("path", po::value(), "Storage path") ; } @@ -713,6 +714,11 @@ int mainEntryClickHouseLocal(int argc, char ** argv) auto code = DB::getCurrentExceptionCode(); return code ? code : 1; } + catch (const boost::program_options::error & e) + { + std::cerr << "Bad arguments: " << e.what() << std::endl; + return DB::ErrorCodes::BAD_ARGUMENTS; + } catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index deb22ca60ef..631d3f2bcc3 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1510,12 +1510,19 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, { cmd_settings.addProgramOptions(options_description.main_description.value()); /// Parse main commandline options. - auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()); - parser.allow_unregistered(); + auto parser = po::command_line_parser(arguments).options(options_description.main_description.value()).allow_unregistered(); po::parsed_options parsed = parser.run(); - auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); + + /// Check unrecognized options without positional options. + auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); + if (!unrecognized_options.empty()) + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); + + /// Check positional options (options after ' -- ', ex: clickhouse-client -- ). + unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); if (unrecognized_options.size() > 1) - throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[1]); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Positional options are not supported."); + po::store(parsed, options); } diff --git a/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh b/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh index d63765fc179..c1d5c357308 100755 --- a/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh +++ b/tests/queries/0_stateless/01527_clickhouse_local_optimize.sh @@ -10,6 +10,6 @@ rm -rf "${WORKING_FOLDER_01527}" mkdir -p "${WORKING_FOLDER_01527}" # OPTIMIZE was crashing due to lack of temporary volume in local -${CLICKHOUSE_LOCAL} --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" -- --path="${WORKING_FOLDER_01527}" +${CLICKHOUSE_LOCAL} --query "drop database if exists d; create database d; create table d.t engine MergeTree order by a as select 1 a; optimize table d.t final" --path="${WORKING_FOLDER_01527}" rm -rf "${WORKING_FOLDER_01527}" diff --git a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh index 8684582ad45..95ecbf09cf5 100755 --- a/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh +++ b/tests/queries/0_stateless/01528_clickhouse_local_prepare_parts.sh @@ -36,10 +36,10 @@ ATTACH TABLE local.data_csv (id UInt64, d Date, s String) Engine=File(CSV, '${WO EOF ## feed the table -${CLICKHOUSE_LOCAL} --query "INSERT INTO local.test SELECT * FROM local.data_csv;" -- --path="${WORKING_FOLDER_01528}" +${CLICKHOUSE_LOCAL} --query "INSERT INTO local.test SELECT * FROM local.data_csv;" --path="${WORKING_FOLDER_01528}" ## check the parts were created -${CLICKHOUSE_LOCAL} --query "SELECT * FROM local.test WHERE id < 10 ORDER BY id;" -- --path="${WORKING_FOLDER_01528}" +${CLICKHOUSE_LOCAL} --query "SELECT * FROM local.test WHERE id < 10 ORDER BY id;" --path="${WORKING_FOLDER_01528}" ################# @@ -49,36 +49,36 @@ cat < "${WORKING_FOLDER_01528}/metadata/local/stdin.sql" ATTACH TABLE local.stdin (id UInt64, d Date, s String) Engine=File(CSV, stdin); EOF -cat <&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_LOCAL} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_LOCAL} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_LOCAL} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_LOCAL} -f 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_LOCAL} --query 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" + + +${CLICKHOUSE_CLIENT} --unknown-option 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_CLIENT} --unknown-option-1 --unknown-option-2 2>&1 | grep -F -q "UNRECOGNIZED_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_CLIENT} -- --unknown-option 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_CLIENT} -- 'positional-argument' 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_CLIENT} --j 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" + +${CLICKHOUSE_CLIENT} --query 2>&1 | grep -F -q "Bad arguments" && echo "OK" || echo "FAIL" + + + diff --git a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference deleted file mode 100644 index 2c4cf540812..00000000000 --- a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.reference +++ /dev/null @@ -1,2 +0,0 @@ -Code: 552. DB::Exception: Unrecognized option '--unknown-option'. (UNRECOGNIZED_ARGUMENTS) -Code: 552. DB::Exception: Unrecognized option '--unknown-option'. (UNRECOGNIZED_ARGUMENTS) diff --git a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh b/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh deleted file mode 100755 index 2fabc761d4c..00000000000 --- a/tests/queries/0_stateless/02096_unknown_option_in_clickhouse_local.sh +++ /dev/null @@ -1,9 +0,0 @@ -#!/usr/bin/env bash -# shellcheck disable=SC2206 - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -${CLICKHOUSE_LOCAL} --unknown-option 2>&1 echo -${CLICKHOUSE_CLIENT} --unknown-option 2>&1 echo From f3c568fbe65dd07f476e11bd84d46143778729d4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:33:10 +0300 Subject: [PATCH 711/950] PolygonDictionary fix bytes_allocated --- src/Dictionaries/PolygonDictionary.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 7a34a9c2b25..8d0f0f1abc4 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -248,6 +248,9 @@ void IPolygonDictionary::calculateBytesAllocated() { /// Index allocated by subclass not counted because it take a small part in relation to attributes and polygons + if (configuration.store_polygon_key_column) + bytes_allocated += key_attribute_column->allocatedBytes(); + for (const auto & column : attributes_columns) bytes_allocated += column->allocatedBytes(); From 8dde0a5dfbc457af33ea9c9b4ce6f15465e62185 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:35:20 +0300 Subject: [PATCH 712/950] ComplexKeyHashedDictionary fix keys copy --- src/Dictionaries/HashedDictionary.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1f3821096da..eec7cec0285 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -667,7 +667,7 @@ Pipe HashedDictionary::read(const Names & column_na if constexpr (dictionary_key_type == DictionaryKeyType::Simple) return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); else - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } template From 574970a852423dc88a254d162dc50994ab4a4c9b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 15 Oct 2021 18:37:38 +0300 Subject: [PATCH 713/950] Update clickhouse-v21.10-released.md --- website/blog/en/2021/clickhouse-v21.10-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.10-released.md b/website/blog/en/2021/clickhouse-v21.10-released.md index 5b6f6d8b9e3..b49bb284f90 100644 --- a/website/blog/en/2021/clickhouse-v21.10-released.md +++ b/website/blog/en/2021/clickhouse-v21.10-released.md @@ -15,7 +15,7 @@ Let's highlight some of these new exciting new capabilities in 21.10: * Instead of logging every query (which can be a lot of logs!), you can now log a random sample of your queries. The number of queries logged is determined by defining a specified probability between 0.0 (no queries logged) and 1.0 (all queries logged) using the new `log_queries_probability` setting. * Positional arguments are now available in your GROUP BY, ORDER BY and LIMIT BY clauses. For example, `SELECT foo, bar, baz FROM my_table ORDER BY 2,3` orders the results by whatever the bar and baz columns (no need to specify column names twice!) -We're always listening for new ideas, and we're happy to welcome new contributors to the ClickHouse project. Whether for submitting code or improving our documentation and examples, please get involved by sending us a pull request or submitting an issue. Our beginner developers contribution guide will help you get started [[https://clickhouse.com/docs/en/development/developer-instruction/](https://clickhouse.com/docs/en/development/developer-instruction/)] +We're always listening for new ideas, and we're happy to welcome new contributors to the ClickHouse project. Whether for submitting code or improving our documentation and examples, please get involved by sending us a pull request or submitting an issue. Our beginner developers contribution guide will help you get started: [https://clickhouse.com/docs/en/development/developer-instruction/](https://clickhouse.com/docs/en/development/developer-instruction/) ## ClickHouse Release Notes From c9638bacbf625fd8c08667175e42428add6fbc1b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:53:17 +0300 Subject: [PATCH 714/950] Added HashedArray dictionary --- src/Dictionaries/CMakeLists.txt | 1 + src/Dictionaries/HashedArrayDictionary.cpp | 689 ++++++++++++++++++ src/Dictionaries/HashedArrayDictionary.h | 211 ++++++ src/Dictionaries/registerDictionaries.cpp | 2 + tests/performance/hashed_array_dictionary.xml | 126 ++++ ...shed_array_dictionary_simple_key.reference | 66 ++ ...098_hashed_array_dictionary_simple_key.sql | 125 ++++ ...hed_array_dictionary_complex_key.reference | 56 ++ 8 files changed, 1276 insertions(+) create mode 100644 src/Dictionaries/HashedArrayDictionary.cpp create mode 100644 src/Dictionaries/HashedArrayDictionary.h create mode 100644 tests/performance/hashed_array_dictionary.xml create mode 100644 tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference create mode 100644 tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql create mode 100644 tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference diff --git a/src/Dictionaries/CMakeLists.txt b/src/Dictionaries/CMakeLists.txt index bc5f0dc9567..b1b3d6d55e0 100644 --- a/src/Dictionaries/CMakeLists.txt +++ b/src/Dictionaries/CMakeLists.txt @@ -10,6 +10,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW set_source_files_properties( FlatDictionary.cpp HashedDictionary.cpp + HashedArrayDictionary.cpp CacheDictionary.cpp RangeHashedDictionary.cpp DirectDictionary.cpp diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp new file mode 100644 index 00000000000..7160cb67e10 --- /dev/null +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -0,0 +1,689 @@ +#include "HashedArrayDictionary.h" + +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int DICTIONARY_IS_EMPTY; + extern const int UNSUPPORTED_METHOD; +} + +template +HashedArrayDictionary::HashedArrayDictionary( + const StorageID & dict_id_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const HashedArrayDictionaryStorageConfiguration & configuration_, + BlockPtr update_field_loaded_block_) + : IDictionary(dict_id_) + , dict_struct(dict_struct_) + , source_ptr(std::move(source_ptr_)) + , configuration(configuration_) + , update_field_loaded_block(std::move(update_field_loaded_block_)) +{ + createAttributes(); + loadData(); + calculateBytesAllocated(); +} + +template +ColumnPtr HashedArrayDictionary::getColumn( + const std::string & attribute_name, + const DataTypePtr & result_type, + const Columns & key_columns, + const DataTypes & key_types [[maybe_unused]], + const ColumnPtr & default_values_column) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + dict_struct.validateKeyTypes(key_types); + + ColumnPtr result; + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); + + const size_t size = extractor.getKeysSize(); + + const auto & dictionary_attribute = dict_struct.getAttribute(attribute_name, result_type); + const size_t attribute_index = dict_struct.attribute_name_to_index.find(attribute_name)->second; + auto & attribute = attributes[attribute_index]; + + bool is_attribute_nullable = attribute.is_index_null.has_value(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + if (attribute.is_index_null) + { + col_null_map_to = ColumnUInt8::create(size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + using ColumnProvider = DictionaryAttributeColumnProvider; + + DictionaryDefaultValueExtractor default_value_extractor(dictionary_attribute.null_value, default_values_column); + + auto column = ColumnProvider::getColumn(dictionary_attribute, size); + + if constexpr (std::is_same_v) + { + auto * out = column.get(); + + getItemsImpl( + attribute, + extractor, + [&](const size_t, const Array & value, bool) { out->insert(value); }, + default_value_extractor); + } + else if constexpr (std::is_same_v) + { + auto * out = column.get(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + extractor, + [&](size_t row, const StringRef value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out->insertData(value.data, value.size); + }, + default_value_extractor); + else + getItemsImpl( + attribute, + extractor, + [&](size_t, const StringRef value, bool) { out->insertData(value.data, value.size); }, + default_value_extractor); + } + else + { + auto & out = column->getData(); + + if (is_attribute_nullable) + getItemsImpl( + attribute, + extractor, + [&](size_t row, const auto value, bool is_null) + { + (*vec_null_map_to)[row] = is_null; + out[row] = value; + }, + default_value_extractor); + else + getItemsImpl( + attribute, + extractor, + [&](size_t row, const auto value, bool) { out[row] = value; }, + default_value_extractor); + } + + result = std::move(column); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (is_attribute_nullable) + result = ColumnNullable::create(std::move(result), std::move(col_null_map_to)); + + return result; +} + +template +ColumnUInt8::Ptr HashedArrayDictionary::hasKeys(const Columns & key_columns, const DataTypes & key_types) const +{ + if (dictionary_key_type == DictionaryKeyType::Complex) + dict_struct.validateKeyTypes(key_types); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor extractor(key_columns, arena_holder.getComplexKeyArena()); + + size_t keys_size = extractor.getKeysSize(); + + auto result = ColumnUInt8::create(keys_size, false); + auto & out = result->getData(); + + if (attributes.empty()) + { + query_count.fetch_add(keys_size, std::memory_order_relaxed); + return result; + } + + size_t keys_found = 0; + + for (size_t requested_key_index = 0; requested_key_index < keys_size; ++requested_key_index) + { + auto requested_key = extractor.extractCurrentKey(); + + out[requested_key_index] = key_attribute.container.find(requested_key) != key_attribute.container.end(); + + keys_found += out[requested_key_index]; + extractor.rollbackCurrentKey(); + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; +} + +template +ColumnPtr HashedArrayDictionary::getHierarchy(ColumnPtr key_column [[maybe_unused]], const DataTypePtr &) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const auto & key_attribute_container = key_attribute.container; + + const UInt64 null_value = dictionary_attribute.null_value.template get(); + const AttributeContainerType & parent_keys_container = std::get>(hierarchical_attribute.container); + + auto is_key_valid_func = [&](auto & key) { return key_attribute_container.find(key) != key_attribute_container.end(); }; + + size_t keys_found = 0; + + auto get_parent_func = [&](auto & hierarchy_key) + { + std::optional result; + + auto it = key_attribute_container.find(hierarchy_key); + + if (it != key_attribute_container.end()) + result = parent_keys_container[it->getMapped()]; + + keys_found += result.has_value(); + + return result; + }; + + auto dictionary_hierarchy_array = getKeysHierarchyArray(keys, null_value, is_key_valid_func, get_parent_func); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return dictionary_hierarchy_array; + } + else + { + return nullptr; + } +} + +template +ColumnUInt8::Ptr HashedArrayDictionary::isInHierarchy( + ColumnPtr key_column [[maybe_unused]], + ColumnPtr in_key_column [[maybe_unused]], + const DataTypePtr &) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + PaddedPODArray keys_backup_storage; + const auto & keys = getColumnVectorData(this, key_column, keys_backup_storage); + + PaddedPODArray keys_in_backup_storage; + const auto & keys_in = getColumnVectorData(this, in_key_column, keys_in_backup_storage); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & dictionary_attribute = dict_struct.attributes[hierarchical_attribute_index]; + auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + + const auto & key_attribute_container = key_attribute.container; + + const UInt64 null_value = dictionary_attribute.null_value.template get(); + const AttributeContainerType & parent_keys_container = std::get>(hierarchical_attribute.container); + + auto is_key_valid_func = [&](auto & key) { return key_attribute_container.find(key) != key_attribute_container.end(); }; + + size_t keys_found = 0; + + auto get_parent_func = [&](auto & hierarchy_key) + { + std::optional result; + + auto it = key_attribute_container.find(hierarchy_key); + + if (it != key_attribute_container.end()) + result = parent_keys_container[it->getMapped()]; + + keys_found += result.has_value(); + + return result; + }; + + auto result = getKeysIsInHierarchyColumn(keys, keys_in, null_value, is_key_valid_func, get_parent_func); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; + } + else + { + return nullptr; + } +} + +template +ColumnPtr HashedArrayDictionary::getDescendants( + ColumnPtr key_column [[maybe_unused]], + const DataTypePtr &, + size_t level [[maybe_unused]]) const +{ + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + { + PaddedPODArray keys_backup; + const auto & keys = getColumnVectorData(this, key_column, keys_backup); + + size_t hierarchical_attribute_index = *dict_struct.hierarchical_attribute_index; + + const auto & hierarchical_attribute = attributes[hierarchical_attribute_index]; + const AttributeContainerType & parent_keys_container = std::get>(hierarchical_attribute.container); + + const auto & key_attribute_container = key_attribute.container; + + HashMap index_to_key; + index_to_key.reserve(key_attribute.container.size()); + + for (auto & [key, value] : key_attribute_container) + index_to_key[value] = key; + + HashMap> parent_to_child; + + for (size_t i = 0; i < parent_keys_container.size(); ++i) + { + const auto * it = index_to_key.find(i); + if (it == index_to_key.end()) + continue; + + auto parent_key = it->getMapped(); + auto child_key = parent_keys_container[i]; + parent_to_child[parent_key].emplace_back(child_key); + } + + size_t keys_found = 0; + auto result = getKeysDescendantsArray(keys, parent_to_child, level, keys_found); + + query_count.fetch_add(keys.size(), std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); + + return result; + } + else + { + return nullptr; + } +} + +template +void HashedArrayDictionary::createAttributes() +{ + const auto size = dict_struct.attributes.size(); + attributes.reserve(size); + + for (const auto & dictionary_attribute : dict_struct.attributes) + { + auto type_call = [&, this](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + auto is_index_null = dictionary_attribute.is_nullable ? std::make_optional>() : std::optional>{}; + std::unique_ptr string_arena = std::is_same_v ? std::make_unique() : nullptr; + Attribute attribute{dictionary_attribute.underlying_type, AttributeContainerType(), std::move(is_index_null), std::move(string_arena)}; + attributes.emplace_back(std::move(attribute)); + }; + + callOnDictionaryAttributeType(dictionary_attribute.underlying_type, type_call); + } +} + +template +void HashedArrayDictionary::updateData() +{ + if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) + { + QueryPipeline pipeline(source_ptr->loadUpdatedAll()); + + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + { + /// We are using this to keep saved data if input stream consists of multiple blocks + if (!update_field_loaded_block) + update_field_loaded_block = std::make_shared(block.cloneEmpty()); + + for (size_t attribute_index = 0; attribute_index < block.columns(); ++attribute_index) + { + const IColumn & update_column = *block.getByPosition(attribute_index).column.get(); + MutableColumnPtr saved_column = update_field_loaded_block->getByPosition(attribute_index).column->assumeMutable(); + saved_column->insertRangeFrom(update_column, 0, update_column.size()); + } + } + } + else + { + auto pipe = source_ptr->loadUpdatedAll(); + mergeBlockWithPipe( + dict_struct.getKeysSize(), + *update_field_loaded_block, + std::move(pipe)); + } + + if (update_field_loaded_block) + { + resize(update_field_loaded_block->rows()); + blockToAttributes(*update_field_loaded_block.get()); + } +} + +template +void HashedArrayDictionary::blockToAttributes(const Block & block [[maybe_unused]]) +{ + size_t skip_keys_size_offset = dict_struct.getKeysSize(); + + Columns key_columns; + key_columns.reserve(skip_keys_size_offset); + + /// Split into keys columns and attribute columns + for (size_t i = 0; i < skip_keys_size_offset; ++i) + key_columns.emplace_back(block.safeGetByPosition(i).column); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); + const size_t keys_size = keys_extractor.getKeysSize(); + + Field column_value_to_insert; + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + auto it = key_attribute.container.find(key); + + if (it != key_attribute.container.end()) + { + keys_extractor.rollbackCurrentKey(); + continue; + } + + if constexpr (std::is_same_v) + key = copyKeyInArena(key); + + key_attribute.container.insert({key, element_count}); + + for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) + { + const IColumn & attribute_column = *block.safeGetByPosition(skip_keys_size_offset + attribute_index).column; + auto & attribute = attributes[attribute_index]; + bool attribute_is_nullable = attribute.is_index_null.has_value(); + + attribute_column.get(key_index, column_value_to_insert); + + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using AttributeValueType = DictionaryValueType; + + auto & attribute_container = std::get>(attribute.container); + attribute_container.emplace_back(); + + if (attribute_is_nullable) { + attribute.is_index_null->emplace_back(); + } + + if (attribute_is_nullable && column_value_to_insert.isNull()) + { + (*attribute.is_index_null).back() = true; + return; + } + + if constexpr (std::is_same_v) + { + String & value_to_insert = column_value_to_insert.get(); + size_t value_to_insert_size = value_to_insert.size(); + + const char * string_in_arena = attribute.string_arena->insert(value_to_insert.data(), value_to_insert_size); + + StringRef string_in_arena_reference = StringRef{string_in_arena, value_to_insert_size}; + attribute_container.back() = string_in_arena_reference; + } + else + { + auto value_to_insert = column_value_to_insert.get>(); + attribute_container.back() = value_to_insert; + } + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + } + + ++element_count; + keys_extractor.rollbackCurrentKey(); + } +} + +template +void HashedArrayDictionary::resize(size_t added_rows) +{ + if (unlikely(!added_rows)) + return; + + key_attribute.container.reserve(added_rows); +} + +template +template +void HashedArrayDictionary::getItemsImpl( + const Attribute & attribute, + DictionaryKeysExtractor & keys_extractor, + ValueSetter && set_value [[maybe_unused]], + DefaultValueExtractor & default_value_extractor) const +{ + const auto & key_attribute_container = key_attribute.container; + const auto & attribute_container = std::get>(attribute.container); + const size_t keys_size = keys_extractor.getKeysSize(); + + size_t keys_found = 0; + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + const auto it = key_attribute_container.find(key); + + if (it != key_attribute_container.end()) + { + size_t element_index = it->getMapped(); + + const auto & element = attribute_container[element_index]; + + if constexpr (is_nullable) + set_value(key_index, element, (*attribute.is_index_null)[element_index]); + else + set_value(key_index, element, false); + + ++keys_found; + } + else + { + if constexpr (is_nullable) + set_value(key_index, default_value_extractor[key_index], default_value_extractor.isNullAt(key_index)); + else + set_value(key_index, default_value_extractor[key_index], false); + } + + keys_extractor.rollbackCurrentKey(); + } + + query_count.fetch_add(keys_size, std::memory_order_relaxed); + found_count.fetch_add(keys_found, std::memory_order_relaxed); +} + +template +StringRef HashedArrayDictionary::copyKeyInArena(StringRef key) +{ + size_t key_size = key.size; + char * place_for_key = complex_key_arena.alloc(key_size); + memcpy(reinterpret_cast(place_for_key), reinterpret_cast(key.data), key_size); + StringRef updated_key{place_for_key, key_size}; + return updated_key; +} + +template +void HashedArrayDictionary::loadData() +{ + if (!source_ptr->hasUpdateField()) + { + QueryPipeline pipeline; + pipeline = QueryPipeline(source_ptr->loadAll()); + + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + { + resize(block.rows()); + blockToAttributes(block); + } + } + else + { + updateData(); + } + + if (configuration.require_nonempty && 0 == element_count) + throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, + "{}: dictionary source is empty and 'require_nonempty' property is set.", + full_name); +} + +template +void HashedArrayDictionary::calculateBytesAllocated() +{ + bytes_allocated += attributes.size() * sizeof(attributes.front()); + + bytes_allocated += key_attribute.container.size(); + + for (auto & attribute : attributes) + { + auto type_call = [&](const auto & dictionary_attribute_type) + { + using Type = std::decay_t; + using AttributeType = typename Type::AttributeType; + using ValueType = DictionaryValueType; + + const auto & container = std::get>(attribute.container); + bytes_allocated += sizeof(AttributeContainerType); + + if constexpr (std::is_same_v) + { + /// It is not accurate calculations + bytes_allocated += sizeof(Array) * container.size(); + } + else + { + bytes_allocated += container.allocated_bytes(); + } + + bucket_count = container.capacity(); + + if constexpr (std::is_same_v) + bytes_allocated += sizeof(Arena) + attribute.string_arena->size(); + }; + + callOnDictionaryAttributeType(attribute.type, type_call); + + if (attribute.string_arena) + bytes_allocated += attribute.string_arena->size(); + } + + bytes_allocated += complex_key_arena.size(); + + if (update_field_loaded_block) + bytes_allocated += update_field_loaded_block->allocatedBytes(); +} + +template +Pipe HashedArrayDictionary::read(const Names & column_names, size_t max_block_size) const +{ + PaddedPODArray keys; + + for (auto & [key, value] : key_attribute.container) + keys.emplace_back(key); + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + else + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); +} + +template class HashedArrayDictionary; +template class HashedArrayDictionary; + +void registerDictionaryArrayHashed(DictionaryFactory & factory) +{ + auto create_layout = [](const std::string & full_name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr, + DictionaryKeyType dictionary_key_type) -> DictionaryPtr + { + if (dictionary_key_type == DictionaryKeyType::Simple && dict_struct.key) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is not supported for simple key hashed array dictionary"); + else if (dictionary_key_type == DictionaryKeyType::Complex && dict_struct.id) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is not supported for complex key hashed array dictionary"); + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "{}: elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + full_name); + + const auto dict_id = StorageID::fromDictionaryConfig(config, config_prefix); + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + + HashedArrayDictionaryStorageConfiguration configuration{require_nonempty, dict_lifetime}; + + if (dictionary_key_type == DictionaryKeyType::Simple) + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + else + return std::make_unique>(dict_id, dict_struct, std::move(source_ptr), configuration); + }; + + using namespace std::placeholders; + + factory.registerLayout("hashed_array", + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Simple); }, false); + factory.registerLayout("complex_key_hashed_array", + [=](auto && a, auto && b, auto && c, auto && d, DictionarySourcePtr e, ContextPtr /* global_context */, bool /*created_from_ddl*/){ return create_layout(a, b, c, d, std::move(e), DictionaryKeyType::Complex); }, true); +} + +} diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h new file mode 100644 index 00000000000..053813bdc44 --- /dev/null +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -0,0 +1,211 @@ +#pragma once + +#include +#include +#include +#include + +#include + +#include +#include +#include + +#include +#include +#include +#include + +/** This dictionary stores all attributes in arrays. + * Key is stored in hash table and value is index into attribute array. + */ + +namespace DB +{ + +struct HashedArrayDictionaryStorageConfiguration +{ + const bool require_nonempty; + const DictionaryLifetime lifetime; +}; + +template +class HashedArrayDictionary final : public IDictionary +{ +public: + using KeyType = std::conditional_t; + + HashedArrayDictionary( + const StorageID & dict_id_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const HashedArrayDictionaryStorageConfiguration & configuration_, + BlockPtr update_field_loaded_block_ = nullptr); + + std::string getTypeName() const override + { + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + return "HashedArray"; + else + return "ComplexHashedArray"; + } + + size_t getBytesAllocated() const override { return bytes_allocated; } + + size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } + + double getFoundRate() const override + { + size_t queries = query_count.load(std::memory_order_relaxed); + if (!queries) + return 0; + return static_cast(found_count.load(std::memory_order_relaxed)) / queries; + } + + double getHitRate() const override { return 1.0; } + + size_t getElementCount() const override { return element_count; } + + double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } + + std::shared_ptr clone() const override + { + return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); + } + + const IDictionarySource * getSource() const override { return source_ptr.get(); } + + const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; } + + const DictionaryStructure & getStructure() const override { return dict_struct; } + + bool isInjective(const std::string & attribute_name) const override + { + return dict_struct.getAttribute(attribute_name).injective; + } + + DictionaryKeyType getKeyType() const override { return dictionary_key_type; } + + ColumnPtr getColumn( + const std::string& attribute_name, + const DataTypePtr & result_type, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnPtr & default_values_column) const override; + + ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; + + bool hasHierarchy() const override { return dictionary_key_type == DictionaryKeyType::Simple && dict_struct.hierarchical_attribute_index.has_value(); } + + ColumnPtr getHierarchy(ColumnPtr key_column, const DataTypePtr & hierarchy_attribute_type) const override; + + ColumnUInt8::Ptr isInHierarchy( + ColumnPtr key_column, + ColumnPtr in_key_column, + const DataTypePtr & key_type) const override; + + ColumnPtr getDescendants( + ColumnPtr key_column, + const DataTypePtr & key_type, + size_t level) const override; + + Pipe read(const Names & column_names, size_t max_block_size) const override; + +private: + + using KeyContainerType = std::conditional_t< + dictionary_key_type == DictionaryKeyType::Simple, + HashMap, + HashMapWithSavedHash>>; + + template + using AttributeContainerType = std::conditional_t, std::vector, PaddedPODArray>; + + struct Attribute final + { + AttributeUnderlyingType type; + + std::variant< + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType, + AttributeContainerType> + container; + + std::optional> is_index_null; + std::unique_ptr string_arena; + }; + + struct KeyAttribute final + { + + KeyContainerType container; + + }; + + void createAttributes(); + + void blockToAttributes(const Block & block); + + void updateData(); + + void loadData(); + + void calculateBytesAllocated(); + + template + void getItemsImpl( + const Attribute & attribute, + DictionaryKeysExtractor & keys_extractor, + ValueSetter && set_value, + DefaultValueExtractor & default_value_extractor) const; + + template + void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func); + + template + void getAttributeContainer(size_t attribute_index, GetContainerFunc && get_container_func) const; + + void resize(size_t added_rows); + + StringRef copyKeyInArena(StringRef key); + + const DictionaryStructure dict_struct; + const DictionarySourcePtr source_ptr; + const HashedArrayDictionaryStorageConfiguration configuration; + + std::vector attributes; + + KeyAttribute key_attribute; + + size_t bytes_allocated = 0; + size_t element_count = 0; + size_t bucket_count = 0; + mutable std::atomic query_count{0}; + mutable std::atomic found_count{0}; + + BlockPtr update_field_loaded_block; + Arena complex_key_arena; +}; + +extern template class HashedArrayDictionary; +extern template class HashedArrayDictionary; + +} diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 8d24a6ea979..df191edd1c3 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -28,6 +28,7 @@ void registerDictionaryComplexKeyHashed(DictionaryFactory & factory); void registerDictionaryTrie(DictionaryFactory & factory); void registerDictionaryFlat(DictionaryFactory & factory); void registerDictionaryHashed(DictionaryFactory & factory); +void registerDictionaryArrayHashed(DictionaryFactory & factory); void registerDictionaryCache(DictionaryFactory & factory); void registerDictionaryPolygon(DictionaryFactory & factory); void registerDictionaryDirect(DictionaryFactory & factory); @@ -60,6 +61,7 @@ void registerDictionaries() registerDictionaryTrie(factory); registerDictionaryFlat(factory); registerDictionaryHashed(factory); + registerDictionaryArrayHashed(factory); registerDictionaryCache(factory); registerDictionaryPolygon(factory); registerDictionaryDirect(factory); diff --git a/tests/performance/hashed_array_dictionary.xml b/tests/performance/hashed_array_dictionary.xml new file mode 100644 index 00000000000..a26e654248f --- /dev/null +++ b/tests/performance/hashed_array_dictionary.xml @@ -0,0 +1,126 @@ + + + CREATE TABLE simple_key_hashed_array_dictionary_source_table + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = Memory; + + + + CREATE TABLE complex_key_hashed_array_dictionary_source_table + ( + id UInt64, + id_key String, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) ENGINE = Memory; + + + + CREATE DICTIONARY simple_key_hashed_array_dictionary + ( + id UInt64, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id + SOURCE(CLICKHOUSE(DB 'default' TABLE 'simple_key_hashed_array_dictionary_source_table')) + LAYOUT(HASHED_ARRAY()) + LIFETIME(MIN 0 MAX 1000); + + + + CREATE DICTIONARY complex_key_hashed_array_dictionary + ( + id UInt64, + id_key String, + value_int UInt64, + value_string String, + value_decimal Decimal64(8), + value_string_nullable Nullable(String) + ) + PRIMARY KEY id, id_key + SOURCE(CLICKHOUSE(DB 'default' TABLE 'complex_key_hashed_array_dictionary_source_table')) + LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) + LIFETIME(MIN 0 MAX 1000); + + + + INSERT INTO simple_key_hashed_array_dictionary_source_table + SELECT number, number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 5000000; + + + + INSERT INTO complex_key_hashed_array_dictionary_source_table + SELECT number, toString(number), number, toString(number), toDecimal64(number, 8), toString(number) + FROM system.numbers + LIMIT 5000000; + + + + + column_name + + 'value_int' + 'value_string' + 'value_decimal' + 'value_string_nullable' + + + + + elements_count + + 5000000 + 7500000 + + + + + + WITH rand64() % toUInt64({elements_count}) as key + SELECT dictGet('default.simple_key_hashed_array_dictionary', {column_name}, key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + WITH rand64() % toUInt64({elements_count}) as key + SELECT dictHas('default.simple_key_hashed_array_dictionary', key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + + WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key + SELECT dictGet('default.complex_key_hashed_array_dictionary', {column_name}, key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + WITH (rand64() % toUInt64({elements_count}), toString(rand64() % toUInt64({elements_count}))) as key + SELECT dictHas('default.complex_key_hashed_array_dictionary', key) + FROM system.numbers + LIMIT {elements_count} + FORMAT Null; + + + DROP TABLE IF EXISTS simple_key_hashed_array_dictionary_source_table; + DROP TABLE IF EXISTS complex_key_hashed_array_dictionary_source_table; + + DROP DICTIONARY IF EXISTS simple_key_hashed_array_dictionary; + DROP DICTIONARY IF EXISTS complex_key_hashed_array_dictionary; + + diff --git a/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference new file mode 100644 index 00000000000..6e88bbad146 --- /dev/null +++ b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.reference @@ -0,0 +1,66 @@ +Dictionary hashed_array_dictionary_simple_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 value_second_1 +2 value_2 value_second_2 +Dictionary hashed_array_dictionary_simple_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 value_0 value_second_0 +1 value_1 \N +2 value_2 value_second_2 +Dictionary hashed_array_dictionary_simple_key_hierarchy +dictGet +0 +0 +1 +1 +2 +dictGetHierarchy +[1] +[4,2,1] diff --git a/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql new file mode 100644 index 00000000000..8d792836562 --- /dev/null +++ b/tests/queries/0_stateless/02098_hashed_array_dictionary_simple_key.sql @@ -0,0 +1,125 @@ +DROP TABLE IF EXISTS simple_key_simple_attributes_source_table; +CREATE TABLE simple_key_simple_attributes_source_table +( + id UInt64, + value_first String, + value_second String +) +ENGINE = TinyLog; + +INSERT INTO simple_key_simple_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(1, 'value_1', 'value_second_1'); +INSERT INTO simple_key_simple_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_simple_key_simple_attributes; +CREATE DICTIONARY hashed_array_dictionary_simple_key_simple_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'simple_key_simple_attributes_source_table')) +LAYOUT(HASHED_ARRAY()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Dictionary hashed_array_dictionary_simple_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_simple_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_simple_key_simple_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_simple_key_simple_attributes ORDER BY id; + +DROP DICTIONARY hashed_array_dictionary_simple_key_simple_attributes; + +DROP TABLE simple_key_simple_attributes_source_table; + +DROP TABLE IF EXISTS simple_key_complex_attributes_source_table; +CREATE TABLE simple_key_complex_attributes_source_table +( + id UInt64, + value_first String, + value_second Nullable(String) +) +ENGINE = TinyLog; + +INSERT INTO simple_key_complex_attributes_source_table VALUES(0, 'value_0', 'value_second_0'); +INSERT INTO simple_key_complex_attributes_source_table VALUES(1, 'value_1', NULL); +INSERT INTO simple_key_complex_attributes_source_table VALUES(2, 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_simple_key_complex_attributes; +CREATE DICTIONARY hashed_array_dictionary_simple_key_complex_attributes +( + id UInt64, + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'simple_key_complex_attributes_source_table')) +LAYOUT(HASHED_ARRAY()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Dictionary hashed_array_dictionary_simple_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number) as value_first, + dictGet('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_first', number, toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_simple_key_complex_attributes', 'value_second', number, toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_simple_key_complex_attributes', number) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_simple_key_complex_attributes ORDER BY id; + +DROP DICTIONARY hashed_array_dictionary_simple_key_complex_attributes; +DROP TABLE simple_key_complex_attributes_source_table; + +DROP TABLE IF EXISTS simple_key_hierarchy_table; +CREATE TABLE simple_key_hierarchy_table +( + id UInt64, + parent_id UInt64 +) ENGINE = TinyLog(); + +INSERT INTO simple_key_hierarchy_table VALUES (1, 0); +INSERT INTO simple_key_hierarchy_table VALUES (2, 1); +INSERT INTO simple_key_hierarchy_table VALUES (3, 1); +INSERT INTO simple_key_hierarchy_table VALUES (4, 2); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_simple_key_hierarchy; +CREATE DICTIONARY hashed_array_dictionary_simple_key_hierarchy +( + id UInt64, + parent_id UInt64 HIERARCHICAL +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'simple_key_hierarchy_table')) +LAYOUT(HASHED_ARRAY()) +LIFETIME(MIN 1 MAX 1000); + +SELECT 'Dictionary hashed_array_dictionary_simple_key_hierarchy'; +SELECT 'dictGet'; +SELECT dictGet('hashed_array_dictionary_simple_key_hierarchy', 'parent_id', number) FROM system.numbers LIMIT 5; +SELECT 'dictGetHierarchy'; +SELECT dictGetHierarchy('hashed_array_dictionary_simple_key_hierarchy', toUInt64(1)); +SELECT dictGetHierarchy('hashed_array_dictionary_simple_key_hierarchy', toUInt64(4)); + +DROP DICTIONARY hashed_array_dictionary_simple_key_hierarchy; +DROP TABLE simple_key_hierarchy_table; diff --git a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference new file mode 100644 index 00000000000..ec32fa72b4e --- /dev/null +++ b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.reference @@ -0,0 +1,56 @@ +Dictionary hashed_array_dictionary_complex_key_simple_attributes +dictGet existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 value_second_1 +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 id_key_0 value_0 value_second_0 +1 id_key_1 value_1 value_second_1 +2 id_key_2 value_2 value_second_2 +Dictionary hashed_array_dictionary_complex_key_complex_attributes +dictGet existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGet with non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +value_first_default value_second_default +dictGetOrDefault existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +dictGetOrDefault non existing value +value_0 value_second_0 +value_1 \N +value_2 value_second_2 +default default +dictHas +1 +1 +1 +0 +select all values as input stream +0 id_key_0 value_0 value_second_0 +1 id_key_1 value_1 \N +2 id_key_2 value_2 value_second_2 From 33ef3edcfabd41f2c1968fea08f445a22a730471 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:54:41 +0300 Subject: [PATCH 715/950] Updated tests --- ...99_hashed_array_dictionary_complex_key.sql | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) create mode 100644 tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql diff --git a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql new file mode 100644 index 00000000000..219b4ab9b71 --- /dev/null +++ b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql @@ -0,0 +1,97 @@ +DROP TABLE IF EXISTS complex_key_simple_attributes_source_table; +CREATE TABLE complex_key_simple_attributes_source_table +( + id UInt64, + id_key String, + value_first String, + value_second String +) +ENGINE = TinyLog; + +INSERT INTO complex_key_simple_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); +INSERT INTO complex_key_simple_attributes_source_table VALUES(1, 'id_key_1', 'value_1', 'value_second_1'); +INSERT INTO complex_key_simple_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_complex_key_simple_attributes; +CREATE DICTIONARY hashed_array_dictionary_complex_key_simple_attributes +( + id UInt64, + id_key String, + value_first String DEFAULT 'value_first_default', + value_second String DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(TABLE 'complex_key_simple_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()); + +SELECT 'Dictionary hashed_array_dictionary_complex_key_simple_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_simple_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_complex_key_simple_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_complex_key_simple_attributes ORDER BY (id, id_key); + +DROP DICTIONARY hashed_array_dictionary_complex_key_simple_attributes; + +DROP TABLE complex_key_simple_attributes_source_table; + +DROP TABLE IF EXISTS complex_key_complex_attributes_source_table; +CREATE TABLE complex_key_complex_attributes_source_table +( + id UInt64, + id_key String, + value_first String, + value_second Nullable(String) +) +ENGINE = TinyLog; + +INSERT INTO complex_key_complex_attributes_source_table VALUES(0, 'id_key_0', 'value_0', 'value_second_0'); +INSERT INTO complex_key_complex_attributes_source_table VALUES(1, 'id_key_1', 'value_1', NULL); +INSERT INTO complex_key_complex_attributes_source_table VALUES(2, 'id_key_2', 'value_2', 'value_second_2'); + +DROP DICTIONARY IF EXISTS hashed_array_dictionary_complex_key_complex_attributes; +CREATE DICTIONARY hashed_array_dictionary_complex_key_complex_attributes +( + id UInt64, + id_key String, + + value_first String DEFAULT 'value_first_default', + value_second Nullable(String) DEFAULT 'value_second_default' +) +PRIMARY KEY id, id_key +SOURCE(CLICKHOUSE(TABLE 'complex_key_complex_attributes_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(()); + +SELECT 'Dictionary hashed_array_dictionary_complex_key_complex_attributes'; +SELECT 'dictGet existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGet with non existing value'; +SELECT dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number)))) as value_first, + dictGet('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number)))) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictGetOrDefault existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 3; +SELECT 'dictGetOrDefault non existing value'; +SELECT dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_first', (number, concat('id_key_', toString(number))), toString('default')) as value_first, + dictGetOrDefault('hashed_array_dictionary_complex_key_complex_attributes', 'value_second', (number, concat('id_key_', toString(number))), toString('default')) as value_second FROM system.numbers LIMIT 4; +SELECT 'dictHas'; +SELECT dictHas('hashed_array_dictionary_complex_key_complex_attributes', (number, concat('id_key_', toString(number)))) FROM system.numbers LIMIT 4; +SELECT 'select all values as input stream'; +SELECT * FROM hashed_array_dictionary_complex_key_complex_attributes ORDER BY (id, id_key); + +DROP DICTIONARY hashed_array_dictionary_complex_key_complex_attributes; +DROP TABLE complex_key_complex_attributes_source_table; From e66735e21e14406801caa06c7fd0922d3e8a4ca2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 18:55:28 +0300 Subject: [PATCH 716/950] Fixed style check --- src/Dictionaries/HashedArrayDictionary.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 7160cb67e10..7706f7d6108 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -454,14 +454,15 @@ void HashedArrayDictionary::blockToAttributes(const Block & auto & attribute_container = std::get>(attribute.container); attribute_container.emplace_back(); - if (attribute_is_nullable) { - attribute.is_index_null->emplace_back(); - } - - if (attribute_is_nullable && column_value_to_insert.isNull()) + if (attribute_is_nullable) { - (*attribute.is_index_null).back() = true; - return; + attribute.is_index_null->emplace_back(); + + if (column_value_to_insert.isNull()) + { + (*attribute.is_index_null).back() = true; + return; + } } if constexpr (std::is_same_v) From 01fbd52758c0b72609999e16b4bee34126a6fe8c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 15 Oct 2021 18:52:23 +0200 Subject: [PATCH 717/950] Fix crash with shortcircuit and locardinality in multiIf --- src/Functions/multiIf.cpp | 22 ++++--- ...owcardinality_shortcircuit_crash.reference | 60 +++++++++++++++++++ ...2049_lowcardinality_shortcircuit_crash.sql | 45 ++++++++++++++ 3 files changed, 115 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference create mode 100644 tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 1122d4892c6..3e5242d5f9b 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -124,8 +124,8 @@ public: */ struct Instruction { - const IColumn * condition = nullptr; - const IColumn * source = nullptr; + IColumn::Ptr condition = nullptr; + IColumn::Ptr source = nullptr; bool condition_always_true = false; bool condition_is_nullable = false; @@ -160,15 +160,15 @@ public: } else { - const ColumnWithTypeAndName & cond_col = arguments[i]; + IColumn::Ptr cond_col = arguments[i].column->convertToFullColumnIfLowCardinality(); /// We skip branches that are always false. /// If we encounter a branch that is always true, we can finish. - if (cond_col.column->onlyNull()) + if (cond_col->onlyNull()) continue; - if (const auto * column_const = checkAndGetColumn(*cond_col.column)) + if (const auto * column_const = checkAndGetColumn(*cond_col)) { Field value = column_const->getField(); @@ -181,26 +181,24 @@ public: } else { - if (isColumnNullable(*cond_col.column)) - instruction.condition_is_nullable = true; - - instruction.condition = cond_col.column.get(); + instruction.condition = cond_col; + instruction.condition_is_nullable = instruction.condition->isNullable(); } - instruction.condition_is_short = cond_col.column->size() < arguments[0].column->size(); + instruction.condition_is_short = cond_col->size() < arguments[0].column->size(); } const ColumnWithTypeAndName & source_col = arguments[source_idx]; instruction.source_is_short = source_col.column->size() < arguments[0].column->size(); if (source_col.type->equals(*return_type)) { - instruction.source = source_col.column.get(); + instruction.source = source_col.column; } else { /// Cast all columns to result type. converted_columns_holder.emplace_back(castColumn(source_col, return_type)); - instruction.source = converted_columns_holder.back().get(); + instruction.source = converted_columns_holder.back(); } if (instruction.source && isColumnConst(*instruction.source)) diff --git a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference new file mode 100644 index 00000000000..c84236dce7d --- /dev/null +++ b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.reference @@ -0,0 +1,60 @@ +0 0 +1 1 +2 2 +3 3 +4 40 +5 50 +6 60 +7 70 +8 800 +9 900 +10 1000 +11 1100 +12 12000 +13 13000 +14 14000 +15 15000 +16 160000 +17 170000 +18 180000 +19 190000 +0 0 +1 1 +2 2 +3 3 +4 40 +5 50 +6 60 +7 70 +8 80000 +9 90000 +10 100000 +11 110000 +12 120000 +13 130000 +14 140000 +15 150000 +16 160000 +17 170000 +18 180000 +19 190000 +0 0 +1 1 +2 2 +3 3 +4 40 +5 50 +6 60 +7 70 +8 800 +9 900 +10 1000 +11 1100 +12 12000 +13 13000 +14 14000 +15 15000 +16 160000 +17 170000 +18 180000 +19 190000 diff --git a/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql new file mode 100644 index 00000000000..2a837380250 --- /dev/null +++ b/tests/queries/0_stateless/02049_lowcardinality_shortcircuit_crash.sql @@ -0,0 +1,45 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/30231 +SELECT * +FROM ( + SELECT number, + multiIf( + CAST(number < 4, 'UInt8'), toString(number), + CAST(number < 8, 'LowCardinality(UInt8)'), toString(number * 10), + CAST(number < 12, 'Nullable(UInt8)'), toString(number * 100), + CAST(number < 16, 'LowCardinality(Nullable(UInt8))'), toString(number * 1000), + toString(number * 10000)) as m + FROM system.numbers + LIMIT 20 + ) +ORDER BY number +SETTINGS short_circuit_function_evaluation='enable'; + +SELECT * +FROM ( + SELECT number, + multiIf( + CAST(number < 4, 'UInt8'), toString(number), + CAST(number < 8, 'LowCardinality(UInt8)'), toString(number * 10), + CAST(NULL, 'Nullable(UInt8)'), toString(number * 100), + CAST(NULL, 'LowCardinality(Nullable(UInt8))'), toString(number * 1000), + toString(number * 10000)) as m + FROM system.numbers + LIMIT 20 + ) +ORDER BY number +SETTINGS short_circuit_function_evaluation='enable'; + +SELECT * +FROM ( + SELECT number, + multiIf( + CAST(number < 4, 'UInt8'), toString(number), + CAST(number < 8, 'LowCardinality(UInt8)'), toString(number * 10)::LowCardinality(String), + CAST(number < 12, 'Nullable(UInt8)'), toString(number * 100)::Nullable(String), + CAST(number < 16, 'LowCardinality(Nullable(UInt8))'), toString(number * 1000)::LowCardinality(Nullable(String)), + toString(number * 10000)) as m + FROM system.numbers + LIMIT 20 + ) +ORDER BY number +SETTINGS short_circuit_function_evaluation='enable'; From ec188000b63aa050a9539b331719469e53749dce Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Oct 2021 17:06:20 +0000 Subject: [PATCH 718/950] Style + more uncomment --- src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt | 2 +- .../fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt index f87340b5024..6f6b0d07661 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt +++ b/src/Parsers/fuzzers/codegen_fuzzer/CMakeLists.txt @@ -30,7 +30,7 @@ add_custom_command( "${CURRENT_DIR_IN_BINARY}/out.cpp" "${CURRENT_DIR_IN_BINARY}/out.proto" COMMAND python3 gen.py clickhouse.g out.cpp out.proto - # DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" + DEPENDS "${CURRENT_DIR_IN_BINARY}/clickhouse.g" ) PROTOBUF_GENERATE_CPP(PROTO_SRCS PROTO_HDRS "${CURRENT_DIR_IN_BINARY}/out.proto") diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 418a5014657..2bc7ad02e87 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -14,13 +14,14 @@ void GenerateSentence(const Sentence&, std::string &, int); -DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) { +DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) +{ static std::string input; input.reserve(4096); GenerateSentence(main, input, 0); - if (input.size()) { - + if (input.size()) + { std::cout << input << std::endl; DB::ParserQueryWithOutput parser(input.data() + input.size()); @@ -30,9 +31,8 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) { DB::WriteBufferFromOStream out(std::cerr, 4096); DB::formatAST(*ast, out); std::cerr << std::endl; - } catch (...) { - } + catch (...) {} input.clear(); } From 882c876090ae88adeaf67db0e6ec1b47f4a2234b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Oct 2021 21:21:08 +0300 Subject: [PATCH 719/950] Update codegen_select_fuzzer.cpp --- src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 2bc7ad02e87..6a7a88a8545 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -25,7 +25,8 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) std::cout << input << std::endl; DB::ParserQueryWithOutput parser(input.data() + input.size()); - try { + try + { DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); DB::WriteBufferFromOStream out(std::cerr, 4096); From f31bf32687257134a66571ab81cb03cfce4c2be8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Oct 2021 22:37:25 +0300 Subject: [PATCH 720/950] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 718aa751cc2..adaaa0f1bc7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,4 @@ -### ClickHouse release v21.10, 2021-10-14 +### ClickHouse release v21.10, 2021-10-16 #### Backward Incompatible Change From fceb763228e1268a5589db1a831ebcf929d28524 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:00:30 +0300 Subject: [PATCH 721/950] Fixed test --- .../0_stateless/02099_hashed_array_dictionary_complex_key.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql index 219b4ab9b71..4d2a825c8af 100644 --- a/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql +++ b/tests/queries/0_stateless/02099_hashed_array_dictionary_complex_key.sql @@ -73,7 +73,7 @@ CREATE DICTIONARY hashed_array_dictionary_complex_key_complex_attributes PRIMARY KEY id, id_key SOURCE(CLICKHOUSE(TABLE 'complex_key_complex_attributes_source_table')) LIFETIME(MIN 1 MAX 1000) -LAYOUT(()); +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()); SELECT 'Dictionary hashed_array_dictionary_complex_key_complex_attributes'; SELECT 'dictGet existing value'; From e4ae49e2f4f15d766cb11cbd86f1bd4ac61b6d91 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:16:56 +0300 Subject: [PATCH 722/950] ComplexKeyHashedDictionary fix config parsing --- src/Dictionaries/HashedDictionary.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 1f3821096da..234f14a661c 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -731,8 +731,18 @@ void registerDictionaryHashed(DictionaryFactory & factory) const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - const std::string & layout_prefix = sparse ? ".layout.sparse_hashed" : ".layout.hashed"; - const bool preallocate = config.getBool(config_prefix + layout_prefix + ".preallocate", false); + std::string dictionary_layout_name; + + if (dictionary_key_type == DictionaryKeyType::Simple) + dictionary_layout_name = "hashed"; + else + dictionary_layout_name = "complex_key_hashed"; + + if (sparse) + dictionary_layout_name = "sparse_" + dictionary_layout_name; + + const std::string dictionary_layout_prefix = ".layout." + dictionary_layout_name; + const bool preallocate = config.getBool(config_prefix + dictionary_layout_prefix + ".preallocate", false); HashedDictionaryStorageConfiguration configuration{preallocate, require_nonempty, dict_lifetime}; From fd14faeae22772322346cedda585a2665d9f866d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 23:18:20 +0300 Subject: [PATCH 723/950] Remove DataStreams folder. --- programs/benchmark/Benchmark.cpp | 4 +-- programs/copier/Internals.h | 2 +- programs/server/Server.cpp | 2 +- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/CMakeLists.txt | 4 +-- src/Client/ClientBase.cpp | 5 ++- src/Client/ClientBase.h | 2 +- src/Client/Connection.cpp | 8 ++--- src/Client/Connection.h | 2 +- src/Client/IServerConnection.h | 4 +-- .../InternalTextLogs.cpp | 2 +- .../InternalTextLogs.h | 0 src/Client/LocalConnection.h | 2 +- .../SquashingTransform.cpp | 2 +- .../SquashingTransform.h | 0 src/Core/Block.cpp | 23 +++++++++++++ src/Core/Block.h | 4 +++ src/Core/SettingsEnums.h | 2 +- src/DataStreams/finalizeBlock.cpp | 27 ---------------- src/DataStreams/finalizeBlock.h | 9 ------ src/DataStreams/materializeBlock.cpp | 29 ----------------- src/DataStreams/materializeBlock.h | 14 -------- src/Databases/DatabaseReplicated.h | 2 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- .../MySQL/FetchTablesColumnsList.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- .../MySQL/MaterializedMySQLSyncThread.h | 2 +- .../ClickHouseDictionarySource.cpp | 2 +- .../ExecutableDictionarySource.cpp | 4 +-- .../ExecutablePoolDictionarySource.cpp | 2 +- .../ExecutablePoolDictionarySource.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.h | 2 +- .../MarkInCompressedFile.h | 0 src/{DataStreams => Formats}/NativeReader.cpp | 2 +- src/{DataStreams => Formats}/NativeReader.h | 2 +- src/{DataStreams => Formats}/NativeWriter.cpp | 4 +-- src/{DataStreams => Formats}/NativeWriter.h | 0 .../TemporaryFileStream.cpp | 29 ++--------------- .../TemporaryFileStream.h | 20 +----------- src/{DataStreams => Formats}/formatBlock.cpp | 2 +- src/{DataStreams => Formats}/formatBlock.h | 0 src/Functions/formatRow.cpp | 1 - src/Interpreters/Aggregator.cpp | 3 +- src/Interpreters/Aggregator.h | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- .../ExecuteScalarSubqueriesVisitor.cpp | 1 - src/Interpreters/HashJoin.cpp | 1 - src/Interpreters/HashJoin.h | 2 +- src/Interpreters/IInterpreter.h | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterExistsQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 -- .../InterpreterShowCreateQuery.cpp | 2 +- src/Interpreters/InterpreterWatchQuery.cpp | 2 +- src/Interpreters/InterpreterWatchQuery.h | 2 +- src/Interpreters/MergeJoin.cpp | 3 +- src/Interpreters/MergeJoin.h | 2 +- src/Interpreters/ProcessList.h | 2 +- src/Interpreters/Set.h | 2 +- src/Interpreters/SortedBlocksWriter.cpp | 4 +-- src/Interpreters/SortedBlocksWriter.h | 2 +- src/Interpreters/TableJoin.h | 2 +- .../UserDefinedExecutableFunction.cpp | 4 +-- .../UserDefinedExecutableFunction.h | 2 +- .../UserDefinedExecutableFunctionFactory.cpp | 4 +-- src/Interpreters/executeDDLQueryOnCluster.h | 2 +- src/Interpreters/executeQuery.cpp | 3 +- src/Interpreters/executeQuery.h | 2 +- src/Interpreters/join_common.cpp | 2 -- src/NOTICE | 3 -- .../PullingAsyncPipelineExecutor.cpp | 4 +-- .../Executors/PullingAsyncPipelineExecutor.h | 4 +-- .../Executors/PullingPipelineExecutor.cpp | 2 +- .../Executors/PullingPipelineExecutor.h | 4 +-- src/Processors/Formats/IRowInputFormat.h | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 4 +-- src/Processors/Formats/LazyOutputFormat.h | 6 ++-- src/Processors/Formats/PullingOutputFormat.h | 6 ++-- src/Processors/Pipe.h | 2 +- src/Processors/QueryPlan/AggregatingStep.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.h | 2 +- src/Processors/QueryPlan/CubeStep.h | 2 +- src/Processors/QueryPlan/DistinctStep.h | 2 +- src/Processors/QueryPlan/MergeSortingStep.h | 2 +- .../QueryPlan/MergingAggregatedStep.h | 2 +- src/Processors/QueryPlan/MergingSortedStep.h | 2 +- src/Processors/QueryPlan/OffsetStep.h | 2 +- src/Processors/QueryPlan/PartialSortingStep.h | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- src/Processors/QueryPlan/RollupStep.h | 2 +- .../QueryPlan/SettingQuotaAndLimitsStep.h | 2 +- src/Processors/Sinks/RemoteSink.h | 27 ++++++++++++++++ .../Sources}/MySQLSource.cpp | 2 +- .../Sources}/MySQLSource.h | 0 src/Processors/Sources/RemoteSource.cpp | 6 ++-- .../Sources}/SQLiteSource.cpp | 0 .../Sources}/SQLiteSource.h | 0 .../Sources}/ShellCommandSource.h | 0 src/Processors/Sources/SourceWithProgress.h | 2 +- .../Sources/TemporaryFileLazySource.cpp | 32 +++++++++++++++++++ .../Sources/TemporaryFileLazySource.h | 28 ++++++++++++++++ .../Transforms/AggregatingTransform.cpp | 3 +- .../Transforms/CreatingSetsTransform.cpp | 1 - .../Transforms/CreatingSetsTransform.h | 2 +- src/Processors/Transforms/DistinctTransform.h | 2 +- .../Transforms/LimitsCheckingTransform.h | 4 +-- .../Transforms/MaterializingTransform.cpp | 1 - .../Transforms/MergeSortingTransform.cpp | 4 +-- .../Transforms/SortingTransform.cpp | 4 +-- .../Transforms/SquashingChunksTransform.h | 2 +- .../Transforms/TotalsHavingTransform.cpp | 21 +++++++++++- .../getSourceFromASTInsertQuery.cpp | 2 +- .../BlockIO.cpp | 2 +- src/{DataStreams => QueryPipeline}/BlockIO.h | 0 .../CMakeLists.txt | 0 .../ConnectionCollector.cpp | 2 +- .../ConnectionCollector.h | 0 .../ExecutionSpeedLimits.cpp | 2 +- .../ExecutionSpeedLimits.h | 2 +- .../ProfileInfo.cpp} | 16 +++++----- .../ProfileInfo.h} | 6 ++-- .../RemoteInserter.cpp} | 2 +- .../RemoteInserter.h} | 20 ------------ .../RemoteQueryExecutor.cpp | 6 ++-- .../RemoteQueryExecutor.h | 4 +-- .../RemoteQueryExecutorReadContext.cpp | 2 +- .../RemoteQueryExecutorReadContext.h | 0 .../SizeLimits.cpp | 2 +- .../SizeLimits.h | 0 .../StreamLocalLimits.h | 4 +-- .../examples/CMakeLists.txt | 0 .../narrowBlockInputStreams.cpp | 0 .../narrowBlockInputStreams.h | 0 .../gtest_blocks_size_merging_streams.cpp | 0 .../tests/gtest_check_sorted_stream.cpp | 0 src/Server/GRPCServer.cpp | 7 ++-- src/Server/TCPHandler.cpp | 6 ++-- src/Server/TCPHandler.h | 8 ++--- src/Storages/Distributed/DirectoryMonitor.cpp | 4 +-- src/Storages/Distributed/DistributedSink.cpp | 4 +-- src/Storages/MarkCache.h | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../MergeTree/MergeTreeWriteAheadLog.h | 4 +-- src/Storages/MergeTree/MutateTask.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageExecutable.cpp | 1 - src/Storages/StorageExecutable.h | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageS3Cluster.cpp | 4 +-- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSet.cpp | 8 ++--- src/Storages/StorageStripeLog.cpp | 4 +-- src/Storages/getStructureOfRemoteTable.cpp | 2 +- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 2 +- 161 files changed, 311 insertions(+), 337 deletions(-) rename src/{DataStreams => Client}/InternalTextLogs.cpp (99%) rename src/{DataStreams => Client}/InternalTextLogs.h (100%) rename src/{DataStreams => Common}/SquashingTransform.cpp (98%) rename src/{DataStreams => Common}/SquashingTransform.h (100%) delete mode 100644 src/DataStreams/finalizeBlock.cpp delete mode 100644 src/DataStreams/finalizeBlock.h delete mode 100644 src/DataStreams/materializeBlock.cpp delete mode 100644 src/DataStreams/materializeBlock.h rename src/{DataStreams => Formats}/MarkInCompressedFile.h (100%) rename src/{DataStreams => Formats}/NativeReader.cpp (99%) rename src/{DataStreams => Formats}/NativeReader.h (98%) rename src/{DataStreams => Formats}/NativeWriter.cpp (98%) rename src/{DataStreams => Formats}/NativeWriter.h (100%) rename src/{DataStreams => Formats}/TemporaryFileStream.cpp (68%) rename src/{DataStreams => Formats}/TemporaryFileStream.h (63%) rename src/{DataStreams => Formats}/formatBlock.cpp (93%) rename src/{DataStreams => Formats}/formatBlock.h (100%) create mode 100644 src/Processors/Sinks/RemoteSink.h rename src/{Formats => Processors/Sources}/MySQLSource.cpp (99%) rename src/{Formats => Processors/Sources}/MySQLSource.h (100%) rename src/{DataStreams => Processors/Sources}/SQLiteSource.cpp (100%) rename src/{DataStreams => Processors/Sources}/SQLiteSource.h (100%) rename src/{DataStreams => Processors/Sources}/ShellCommandSource.h (100%) create mode 100644 src/Processors/Sources/TemporaryFileLazySource.cpp create mode 100644 src/Processors/Sources/TemporaryFileLazySource.h rename src/{DataStreams => QueryPipeline}/BlockIO.cpp (97%) rename src/{DataStreams => QueryPipeline}/BlockIO.h (100%) rename src/{DataStreams => QueryPipeline}/CMakeLists.txt (100%) rename src/{DataStreams => QueryPipeline}/ConnectionCollector.cpp (98%) rename src/{DataStreams => QueryPipeline}/ConnectionCollector.h (100%) rename src/{DataStreams => QueryPipeline}/ExecutionSpeedLimits.cpp (99%) rename src/{DataStreams => QueryPipeline}/ExecutionSpeedLimits.h (95%) rename src/{DataStreams/BlockStreamProfileInfo.cpp => QueryPipeline/ProfileInfo.cpp} (67%) rename src/{DataStreams/BlockStreamProfileInfo.h => QueryPipeline/ProfileInfo.h} (90%) rename src/{DataStreams/RemoteBlockOutputStream.cpp => QueryPipeline/RemoteInserter.cpp} (98%) rename src/{DataStreams/RemoteBlockOutputStream.h => QueryPipeline/RemoteInserter.h} (56%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutor.cpp (99%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutor.h (98%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutorReadContext.cpp (99%) rename src/{DataStreams => QueryPipeline}/RemoteQueryExecutorReadContext.h (100%) rename src/{DataStreams => QueryPipeline}/SizeLimits.cpp (97%) rename src/{DataStreams => QueryPipeline}/SizeLimits.h (100%) rename src/{DataStreams => QueryPipeline}/StreamLocalLimits.h (91%) rename src/{DataStreams => QueryPipeline}/examples/CMakeLists.txt (100%) rename src/{DataStreams => QueryPipeline}/narrowBlockInputStreams.cpp (100%) rename src/{DataStreams => QueryPipeline}/narrowBlockInputStreams.h (100%) rename src/{DataStreams => QueryPipeline}/tests/gtest_blocks_size_merging_streams.cpp (100%) rename src/{DataStreams => QueryPipeline}/tests/gtest_check_sorted_stream.cpp (100%) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index caa0a87bde2..1c276a83768 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -28,7 +28,7 @@ #include #include #include -#include +#include #include #include #include @@ -432,7 +432,7 @@ private: Progress progress; executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); }); - BlockStreamProfileInfo info; + ProfileInfo info; while (Block block = executor.read()) info.update(block); diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index a9d8ca726fe..eb2622c6b26 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -49,7 +49,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 4ed5b114082..eb4b79e995b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -62,7 +62,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 319c6c344d7..9bc14627ac3 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -1,6 +1,6 @@ #include "LibraryBridgeHelper.h" -#include +#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 08755542ed1..09aaa85c394 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -49,7 +49,7 @@ add_subdirectory (Backups) add_subdirectory (Columns) add_subdirectory (Common) add_subdirectory (Core) -add_subdirectory (DataStreams) +add_subdirectory (QueryPipeline) add_subdirectory (DataTypes) add_subdirectory (Dictionaries) add_subdirectory (Disks) @@ -185,7 +185,7 @@ add_object_library(clickhouse_backups Backups) add_object_library(clickhouse_core Core) add_object_library(clickhouse_core_mysql Core/MySQL) add_object_library(clickhouse_compression Compression) -add_object_library(clickhouse_datastreams DataStreams) +add_object_library(clickhouse_querypipeline QueryPipeline) add_object_library(clickhouse_datatypes DataTypes) add_object_library(clickhouse_datatypes_serializations DataTypes/Serializations) add_object_library(clickhouse_databases Databases) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e260ed11bd4..f8bed86d7ce 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -47,8 +47,7 @@ #include #include #include -#include -#include +#include namespace fs = std::filesystem; @@ -284,7 +283,7 @@ void ClientBase::onReceiveExceptionFromServer(std::unique_ptr && e) } -void ClientBase::onProfileInfo(const BlockStreamProfileInfo & profile_info) +void ClientBase::onProfileInfo(const ProfileInfo & profile_info) { if (profile_info.hasAppliedLimit() && output_format) output_format->setRowsBeforeLimit(profile_info.getRowsBeforeLimit()); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 92f9d48e070..a66e4a0d1ba 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -112,7 +112,7 @@ private: void onTotals(Block & block, ASTPtr parsed_query); void onExtremes(Block & block, ASTPtr parsed_query); void onReceiveExceptionFromServer(std::unique_ptr && e); - void onProfileInfo(const BlockStreamProfileInfo & profile_info); + void onProfileInfo(const ProfileInfo & profile_info); void onEndOfStream(); void sendData(Block & sample, const ColumnsDescription & columns_description, ASTPtr parsed_query); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2aa157bb318..bdaf4ffc76f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -994,9 +994,9 @@ Progress Connection::receiveProgress() const } -BlockStreamProfileInfo Connection::receiveProfileInfo() const +ProfileInfo Connection::receiveProfileInfo() const { - BlockStreamProfileInfo profile_info; + ProfileInfo profile_info; profile_info.read(*in); return profile_info; } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index a5130d876ea..b18b1f1e621 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -253,7 +253,7 @@ private: std::vector receiveMultistringMessage(UInt64 msg_type) const; std::unique_ptr receiveException() const; Progress receiveProgress() const; - BlockStreamProfileInfo receiveProfileInfo() const; + ProfileInfo receiveProfileInfo() const; void initInputBuffers(); void initBlockInput(); diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 42886c72182..7424afc969d 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include @@ -30,7 +30,7 @@ struct Packet std::unique_ptr exception; std::vector multistring_message; Progress progress; - BlockStreamProfileInfo profile_info; + ProfileInfo profile_info; std::vector part_uuids; Packet() : type(Protocol::Server::Hello) {} diff --git a/src/DataStreams/InternalTextLogs.cpp b/src/Client/InternalTextLogs.cpp similarity index 99% rename from src/DataStreams/InternalTextLogs.cpp rename to src/Client/InternalTextLogs.cpp index a5883d17f28..65592fee670 100644 --- a/src/DataStreams/InternalTextLogs.cpp +++ b/src/Client/InternalTextLogs.cpp @@ -1,4 +1,4 @@ -#include "InternalTextLogs.h" +#include #include #include #include diff --git a/src/DataStreams/InternalTextLogs.h b/src/Client/InternalTextLogs.h similarity index 100% rename from src/DataStreams/InternalTextLogs.h rename to src/Client/InternalTextLogs.h diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index dcea3ed0fc3..5536aeec964 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -2,7 +2,7 @@ #include "Connection.h" #include -#include +#include #include #include diff --git a/src/DataStreams/SquashingTransform.cpp b/src/Common/SquashingTransform.cpp similarity index 98% rename from src/DataStreams/SquashingTransform.cpp rename to src/Common/SquashingTransform.cpp index ea99dc49780..21fa25ed3af 100644 --- a/src/DataStreams/SquashingTransform.cpp +++ b/src/Common/SquashingTransform.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/DataStreams/SquashingTransform.h b/src/Common/SquashingTransform.h similarity index 100% rename from src/DataStreams/SquashingTransform.h rename to src/Common/SquashingTransform.h diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a59ac60155e..40fc5767132 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -707,4 +707,27 @@ ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & column return current_column; } + +Block materializeBlock(const Block & block) +{ + if (!block) + return block; + + Block res = block; + size_t columns = res.columns(); + for (size_t i = 0; i < columns; ++i) + { + auto & element = res.getByPosition(i); + element.column = element.column->convertToFullColumnIfConst(); + } + + return res; +} + +void materializeBlockInplace(Block & block) +{ + for (size_t i = 0; i < block.columns(); ++i) + block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst(); +} + } diff --git a/src/Core/Block.h b/src/Core/Block.h index a7e3cee194b..e0a032094f6 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -196,4 +196,8 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out /// Properly handles cases, when column is a subcolumn and when it is compressed. ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & column); +/// Converts columns-constants to full columns ("materializes" them). +Block materializeBlock(const Block & block); +void materializeBlockInplace(Block & block); + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index a699da3062c..33c5a6d8645 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/src/DataStreams/finalizeBlock.cpp b/src/DataStreams/finalizeBlock.cpp deleted file mode 100644 index 56068edcc29..00000000000 --- a/src/DataStreams/finalizeBlock.cpp +++ /dev/null @@ -1,27 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - void finalizeBlock(Block & block) - { - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnWithTypeAndName & current = block.getByPosition(i); - const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); - - if (unfinalized_type) - { - current.type = unfinalized_type->getReturnType(); - if (current.column) - { - auto mut_column = IColumn::mutate(std::move(current.column)); - current.column = ColumnAggregateFunction::convertToValues(std::move(mut_column)); - } - } - } - } -} diff --git a/src/DataStreams/finalizeBlock.h b/src/DataStreams/finalizeBlock.h deleted file mode 100644 index 3c81ddae1c7..00000000000 --- a/src/DataStreams/finalizeBlock.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - /// Converts aggregate function columns with non-finalized states to final values - void finalizeBlock(Block & block); -} diff --git a/src/DataStreams/materializeBlock.cpp b/src/DataStreams/materializeBlock.cpp deleted file mode 100644 index 6b47cb87baa..00000000000 --- a/src/DataStreams/materializeBlock.cpp +++ /dev/null @@ -1,29 +0,0 @@ -#include - - -namespace DB -{ - -Block materializeBlock(const Block & block) -{ - if (!block) - return block; - - Block res = block; - size_t columns = res.columns(); - for (size_t i = 0; i < columns; ++i) - { - auto & element = res.getByPosition(i); - element.column = element.column->convertToFullColumnIfConst(); - } - - return res; -} - -void materializeBlockInplace(Block & block) -{ - for (size_t i = 0; i < block.columns(); ++i) - block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst(); -} - -} diff --git a/src/DataStreams/materializeBlock.h b/src/DataStreams/materializeBlock.h deleted file mode 100644 index 5e1499319c1..00000000000 --- a/src/DataStreams/materializeBlock.h +++ /dev/null @@ -1,14 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/** Converts columns-constants to full columns ("materializes" them). - */ -Block materializeBlock(const Block & block); -void materializeBlockInplace(Block & block); - -} diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index aecbc1474f8..21d927dea77 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index a30341ae927..24a285f11c4 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -11,7 +11,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 618f6bf6d34..ab144761e11 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 514978f2456..8de42760dc2 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 9ec8a9523c6..d2c1195c0c5 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -16,7 +16,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.h b/src/Databases/MySQL/MaterializedMySQLSyncThread.h index 0cd0701439f..b8c985915dc 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.h @@ -8,7 +8,7 @@ # include # include -# include +# include # include # include # include diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index b563ac797c0..a19eca1fee7 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 91aeda924a1..c09993c2a84 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -4,8 +4,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 79e9b627836..8d1122b1194 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index b7e8468b815..51215b6311b 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -7,7 +7,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 844a6357e29..16e7f483978 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,5 +1,5 @@ #include "HTTPDictionarySource.h" -#include +#include #include #include #include diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index fa26c2b162a..1ecc41036be 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -12,7 +12,7 @@ # include "DictionaryStructure.h" # include "ExternalQueryBuilder.h" # include "IDictionarySource.h" -# include +# include namespace Poco { diff --git a/src/DataStreams/MarkInCompressedFile.h b/src/Formats/MarkInCompressedFile.h similarity index 100% rename from src/DataStreams/MarkInCompressedFile.h rename to src/Formats/MarkInCompressedFile.h diff --git a/src/DataStreams/NativeReader.cpp b/src/Formats/NativeReader.cpp similarity index 99% rename from src/DataStreams/NativeReader.cpp rename to src/Formats/NativeReader.cpp index 079dff80eae..2d8fdc160f5 100644 --- a/src/DataStreams/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include diff --git a/src/DataStreams/NativeReader.h b/src/Formats/NativeReader.h similarity index 98% rename from src/DataStreams/NativeReader.h rename to src/Formats/NativeReader.h index 95b03c71764..49c2db7703f 100644 --- a/src/DataStreams/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/DataStreams/NativeWriter.cpp b/src/Formats/NativeWriter.cpp similarity index 98% rename from src/DataStreams/NativeWriter.cpp rename to src/Formats/NativeWriter.cpp index 6e26c443e29..9da0c312362 100644 --- a/src/DataStreams/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -5,8 +5,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/src/DataStreams/NativeWriter.h b/src/Formats/NativeWriter.h similarity index 100% rename from src/DataStreams/NativeWriter.h rename to src/Formats/NativeWriter.h diff --git a/src/DataStreams/TemporaryFileStream.cpp b/src/Formats/TemporaryFileStream.cpp similarity index 68% rename from src/DataStreams/TemporaryFileStream.cpp rename to src/Formats/TemporaryFileStream.cpp index 4b7c9d50fe7..b19c4aeff35 100644 --- a/src/DataStreams/TemporaryFileStream.cpp +++ b/src/Formats/TemporaryFileStream.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include #include @@ -41,27 +41,4 @@ void TemporaryFileStream::write(const std::string & path, const Block & header, compressed_buf.finalize(); } -TemporaryFileLazySource::TemporaryFileLazySource(const std::string & path_, const Block & header_) - : ISource(header_) - , path(path_) - , done(false) -{} - -Chunk TemporaryFileLazySource::generate() -{ - if (done) - return {}; - - if (!stream) - stream = std::make_unique(path, header); - - auto block = stream->block_in->read(); - if (!block) - { - done = true; - stream.reset(); - } - return Chunk(block.getColumns(), block.rows()); -} - } diff --git a/src/DataStreams/TemporaryFileStream.h b/src/Formats/TemporaryFileStream.h similarity index 63% rename from src/DataStreams/TemporaryFileStream.h rename to src/Formats/TemporaryFileStream.h index e288b5b30fa..5a1e0bc870a 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/Formats/TemporaryFileStream.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { @@ -23,22 +23,4 @@ struct TemporaryFileStream static void write(const std::string & path, const Block & header, QueryPipelineBuilder builder, const std::string & codec); }; - -class TemporaryFileLazySource : public ISource -{ -public: - TemporaryFileLazySource(const std::string & path_, const Block & header_); - String getName() const override { return "TemporaryFileLazySource"; } - -protected: - Chunk generate() override; - -private: - const std::string path; - Block header; - bool done; - - std::unique_ptr stream; -}; - } diff --git a/src/DataStreams/formatBlock.cpp b/src/Formats/formatBlock.cpp similarity index 93% rename from src/DataStreams/formatBlock.cpp rename to src/Formats/formatBlock.cpp index dab321be2e1..3284663fc42 100644 --- a/src/DataStreams/formatBlock.cpp +++ b/src/Formats/formatBlock.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/DataStreams/formatBlock.h b/src/Formats/formatBlock.h similarity index 100% rename from src/DataStreams/formatBlock.h rename to src/Formats/formatBlock.h diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 20341cbe1dc..ee9696cf34f 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 63e3577af55..4f4b981b44d 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -9,8 +9,7 @@ #include #include #include -#include -#include +#include #include #include #include diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 975075eba96..6d6bf61834b 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -19,7 +19,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index b4ce9f352a2..7e0fa2ba003 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 2b858512b98..2117eec0063 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 156bbfc2d81..fd1c10e8495 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -23,7 +23,6 @@ #include -#include #include #include diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 07fd6d5b89f..b5790c047f4 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -20,7 +20,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 1b4eada3c9f..665a46190fd 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 2ebae17cd6b..3cd39ce7912 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 6ffeef5cc7d..24c30a8be30 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index c29eace1b55..487fa2538c2 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 5f44603a420..e5733a8c28b 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b4ffa15a869..59fd1009381 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,5 +1,3 @@ -#include - #include #include diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index c191a73bc71..30a417f6fa7 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index 78c4eca5ca6..e5e447562c6 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -15,7 +15,7 @@ limitations under the License. */ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/InterpreterWatchQuery.h b/src/Interpreters/InterpreterWatchQuery.h index e43ed88af2f..ac167182a71 100644 --- a/src/Interpreters/InterpreterWatchQuery.h +++ b/src/Interpreters/InterpreterWatchQuery.h @@ -12,7 +12,7 @@ See the License for the specific language governing permissions and limitations under the License. */ #include -#include +#include #include #include #include diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 1fc551334e2..7789c74d596 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -4,8 +4,7 @@ #include #include -#include -#include +#include #include #include #include diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 8782a2f7535..0e2e771255d 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 2e300472647..9597c1ee558 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 118779f1935..3146b6af03f 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index ebe4aba71ab..ebec58dcca7 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Interpreters/SortedBlocksWriter.h b/src/Interpreters/SortedBlocksWriter.h index 94bebce88f7..ac58ef2ab7b 100644 --- a/src/Interpreters/SortedBlocksWriter.h +++ b/src/Interpreters/SortedBlocksWriter.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 02dcd95ab41..7cd53442ffd 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/UserDefinedExecutableFunction.cpp b/src/Interpreters/UserDefinedExecutableFunction.cpp index 06830df68e6..d57978d0fd6 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.cpp +++ b/src/Interpreters/UserDefinedExecutableFunction.cpp @@ -4,8 +4,8 @@ #include #include -#include -#include +#include +#include namespace DB diff --git a/src/Interpreters/UserDefinedExecutableFunction.h b/src/Interpreters/UserDefinedExecutableFunction.h index 240422a02ca..1cb1de47578 100644 --- a/src/Interpreters/UserDefinedExecutableFunction.h +++ b/src/Interpreters/UserDefinedExecutableFunction.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index d6ad2666ff1..cfa1171a84b 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -2,8 +2,8 @@ #include -#include -#include +#include +#include #include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.h b/src/Interpreters/executeDDLQueryOnCluster.h index 0ad40dd3332..e7ec52d03cb 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.h +++ b/src/Interpreters/executeDDLQueryOnCluster.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 0a1130c721b..95fb8d38454 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include @@ -49,7 +49,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index a2df9baec73..9c561d8b88c 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index dec925d68c1..bf5d30437ec 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -4,8 +4,6 @@ #include #include -#include - #include #include #include diff --git a/src/NOTICE b/src/NOTICE index d0d3efe3f8e..c68280b1529 100644 --- a/src/NOTICE +++ b/src/NOTICE @@ -18,9 +18,6 @@ Common/UInt128.h Core/Block.h Core/Defines.h Core/Settings.h -DataStreams/PushingToViewsBlockOutputStream.cpp -DataStreams/PushingToViewsBlockOutputStream.h -DataStreams/copyData.cpp Databases/DatabasesCommon.cpp IO/WriteBufferValidUTF8.cpp Interpreters/InterpreterAlterQuery.cpp diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 819344e4225..80ed8225c79 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -225,12 +225,12 @@ Block PullingAsyncPipelineExecutor::getExtremesBlock() return header.cloneWithColumns(extremes.detachColumns()); } -BlockStreamProfileInfo & PullingAsyncPipelineExecutor::getProfileInfo() +ProfileInfo & PullingAsyncPipelineExecutor::getProfileInfo() { if (lazy_format) return lazy_format->getProfileInfo(); - static BlockStreamProfileInfo profile_info; + static ProfileInfo profile_info; static std::once_flag flag; /// Calculate rows before limit here to avoid race. std::call_once(flag, []() { profile_info.getRowsBeforeLimit(); }); diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 2ce75aecab7..7e45246ffd6 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -8,7 +8,7 @@ class QueryPipeline; class Block; class Chunk; class LazyOutputFormat; -struct BlockStreamProfileInfo; +struct ProfileInfo; /// Asynchronous pulling executor for QueryPipeline. /// Always creates extra thread. If query is executed in single thread, use PullingPipelineExecutor. @@ -44,7 +44,7 @@ public: Block getExtremesBlock(); /// Get query profile info. - BlockStreamProfileInfo & getProfileInfo(); + ProfileInfo & getProfileInfo(); /// Internal executor data. struct Data; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 7da2a6d3059..ad7da63b0b1 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -118,7 +118,7 @@ Block PullingPipelineExecutor::getExtremesBlock() return header.cloneWithColumns(extremes.detachColumns()); } -BlockStreamProfileInfo & PullingPipelineExecutor::getProfileInfo() +ProfileInfo & PullingPipelineExecutor::getProfileInfo() { return pulling_format->getProfileInfo(); } diff --git a/src/Processors/Executors/PullingPipelineExecutor.h b/src/Processors/Executors/PullingPipelineExecutor.h index 878d66bd3d4..e05f4f3738d 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.h +++ b/src/Processors/Executors/PullingPipelineExecutor.h @@ -10,7 +10,7 @@ class Chunk; class QueryPipeline; class PipelineExecutor; class PullingOutputFormat; -struct BlockStreamProfileInfo; +struct ProfileInfo; using PipelineExecutorPtr = std::shared_ptr; @@ -46,7 +46,7 @@ public: Block getExtremesBlock(); /// Get query profile info. - BlockStreamProfileInfo & getProfileInfo(); + ProfileInfo & getProfileInfo(); private: std::atomic_bool has_data_flag = false; diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 19a94d41044..87caadd93da 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include class Stopwatch; diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 5daa38967b9..07cf4670981 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 2c29f55c4f3..82a0cb2fc07 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include namespace DB @@ -25,7 +25,7 @@ public: bool isFinished() { return finished_processing && queue.size() == 0; } - BlockStreamProfileInfo & getProfileInfo() { return info; } + ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; @@ -65,7 +65,7 @@ private: /// Is not used. static WriteBuffer out; - BlockStreamProfileInfo info; + ProfileInfo info; std::atomic finished_processing; }; diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index 53b2086712f..a231b7679f3 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { @@ -20,7 +20,7 @@ public: Chunk getTotals(); Chunk getExtremes(); - BlockStreamProfileInfo & getProfileInfo() { return info; } + ProfileInfo & getProfileInfo() { return info; } void setRowsBeforeLimit(size_t rows_before_limit) override; @@ -38,7 +38,7 @@ private: std::atomic_bool & has_data_flag; - BlockStreamProfileInfo info; + ProfileInfo info; /// Is not used. static WriteBuffer out; diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index a07c68f56b2..3341734430c 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 8583e5be485..c7d67c75894 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 8d20c764e8a..e20c28e10f4 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/CubeStep.h b/src/Processors/QueryPlan/CubeStep.h index 45077d78a90..1079bed5398 100644 --- a/src/Processors/QueryPlan/CubeStep.h +++ b/src/Processors/QueryPlan/CubeStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index b08e93dffa9..a48a779425d 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h index d5daa041256..947ced829c6 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ b/src/Processors/QueryPlan/MergeSortingStep.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 9171512571a..eeead41b5f9 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h index 5d27e59ab76..e886de42ca8 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ b/src/Processors/QueryPlan/MergingSortedStep.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/OffsetStep.h b/src/Processors/QueryPlan/OffsetStep.h index 488c55b6460..f16559bcfad 100644 --- a/src/Processors/QueryPlan/OffsetStep.h +++ b/src/Processors/QueryPlan/OffsetStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h index bd8fd30ce02..9b7b8e8baa5 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ b/src/Processors/QueryPlan/PartialSortingStep.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cd2f42ece58..399e7d01839 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/RollupStep.h b/src/Processors/QueryPlan/RollupStep.h index 2ff3040d7a7..7cd71fecdc1 100644 --- a/src/Processors/QueryPlan/RollupStep.h +++ b/src/Processors/QueryPlan/RollupStep.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h index b36ddfb3768..a8d1eef4b08 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Sinks/RemoteSink.h b/src/Processors/Sinks/RemoteSink.h new file mode 100644 index 00000000000..30cf958c072 --- /dev/null +++ b/src/Processors/Sinks/RemoteSink.h @@ -0,0 +1,27 @@ +#pragma once +#include +#include + +namespace DB +{ + +class RemoteSink final : public RemoteInserter, public SinkToStorage +{ +public: + explicit RemoteSink( + Connection & connection_, + const ConnectionTimeouts & timeouts, + const String & query_, + const Settings & settings_, + const ClientInfo & client_info_) + : RemoteInserter(connection_, timeouts, query_, settings_, client_info_) + , SinkToStorage(RemoteInserter::getHeader()) + { + } + + String getName() const override { return "RemoteSink"; } + void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } + void onFinish() override { RemoteInserter::onFinish(); } +}; + +} diff --git a/src/Formats/MySQLSource.cpp b/src/Processors/Sources/MySQLSource.cpp similarity index 99% rename from src/Formats/MySQLSource.cpp rename to src/Processors/Sources/MySQLSource.cpp index 069aa0cb56b..5bda662466c 100644 --- a/src/Formats/MySQLSource.cpp +++ b/src/Processors/Sources/MySQLSource.cpp @@ -19,7 +19,7 @@ #include #include #include -#include "MySQLSource.h" +#include namespace DB diff --git a/src/Formats/MySQLSource.h b/src/Processors/Sources/MySQLSource.h similarity index 100% rename from src/Formats/MySQLSource.h rename to src/Processors/Sources/MySQLSource.h diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index bf3ef32214d..99ba459cf2c 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -1,6 +1,6 @@ #include -#include -#include +#include +#include #include #include @@ -56,7 +56,7 @@ std::optional RemoteSource::tryGenerate() query_executor->setProgressCallback([this](const Progress & value) { progress(value); }); /// Get rows_before_limit result for remote query from ProfileInfo packet. - query_executor->setProfileInfoCallback([this](const BlockStreamProfileInfo & info) + query_executor->setProfileInfoCallback([this](const ProfileInfo & info) { if (rows_before_limit && info.hasAppliedLimit()) rows_before_limit->set(info.getRowsBeforeLimit()); diff --git a/src/DataStreams/SQLiteSource.cpp b/src/Processors/Sources/SQLiteSource.cpp similarity index 100% rename from src/DataStreams/SQLiteSource.cpp rename to src/Processors/Sources/SQLiteSource.cpp diff --git a/src/DataStreams/SQLiteSource.h b/src/Processors/Sources/SQLiteSource.h similarity index 100% rename from src/DataStreams/SQLiteSource.h rename to src/Processors/Sources/SQLiteSource.h diff --git a/src/DataStreams/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h similarity index 100% rename from src/DataStreams/ShellCommandSource.h rename to src/Processors/Sources/ShellCommandSource.h diff --git a/src/Processors/Sources/SourceWithProgress.h b/src/Processors/Sources/SourceWithProgress.h index bf57c3b013b..912a548f977 100644 --- a/src/Processors/Sources/SourceWithProgress.h +++ b/src/Processors/Sources/SourceWithProgress.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include diff --git a/src/Processors/Sources/TemporaryFileLazySource.cpp b/src/Processors/Sources/TemporaryFileLazySource.cpp new file mode 100644 index 00000000000..0382229a7c0 --- /dev/null +++ b/src/Processors/Sources/TemporaryFileLazySource.cpp @@ -0,0 +1,32 @@ +#include +#include + +namespace DB +{ + +TemporaryFileLazySource::~TemporaryFileLazySource() = default; + +TemporaryFileLazySource::TemporaryFileLazySource(const std::string & path_, const Block & header_) + : ISource(header_) + , path(path_) + , done(false) +{} + +Chunk TemporaryFileLazySource::generate() +{ + if (done) + return {}; + + if (!stream) + stream = std::make_unique(path, header); + + auto block = stream->block_in->read(); + if (!block) + { + done = true; + stream.reset(); + } + return Chunk(block.getColumns(), block.rows()); +} + +} diff --git a/src/Processors/Sources/TemporaryFileLazySource.h b/src/Processors/Sources/TemporaryFileLazySource.h new file mode 100644 index 00000000000..b2e9d5d5500 --- /dev/null +++ b/src/Processors/Sources/TemporaryFileLazySource.h @@ -0,0 +1,28 @@ +#pragma once + +#include + +namespace DB +{ + +struct TemporaryFileStream; + +class TemporaryFileLazySource : public ISource +{ +public: + TemporaryFileLazySource(const std::string & path_, const Block & header_); + ~TemporaryFileLazySource() override; + String getName() const override { return "TemporaryFileLazySource"; } + +protected: + Chunk generate() override; + +private: + const std::string path; + Block header; + bool done; + + std::unique_ptr stream; +}; + +} diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 9011d188b81..bf3cafd6ff5 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -1,10 +1,9 @@ #include -#include +#include #include #include #include -#include #include namespace ProfileEvents diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e6ae620e69b..fb3c8d6a87b 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -7,7 +7,6 @@ #include #include -#include namespace DB diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index a5a67e99afc..8e7a09b320d 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Processors/Transforms/DistinctTransform.h b/src/Processors/Transforms/DistinctTransform.h index 236f9026c63..d80fdb5bc22 100644 --- a/src/Processors/Transforms/DistinctTransform.h +++ b/src/Processors/Transforms/DistinctTransform.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Processors/Transforms/LimitsCheckingTransform.h b/src/Processors/Transforms/LimitsCheckingTransform.h index 9de5cbf5125..50891ece654 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.h +++ b/src/Processors/Transforms/LimitsCheckingTransform.h @@ -1,10 +1,10 @@ #pragma once #include -#include +#include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/MaterializingTransform.cpp b/src/Processors/Transforms/MaterializingTransform.cpp index f13d5376ebe..abf416e8047 100644 --- a/src/Processors/Transforms/MaterializingTransform.cpp +++ b/src/Processors/Transforms/MaterializingTransform.cpp @@ -1,5 +1,4 @@ #include -#include namespace DB { diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 6e379a3c4ba..e272fd0f183 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -6,8 +6,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 2c9098adaa6..eeb576731ab 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -9,8 +9,8 @@ #include #include -#include -#include +#include +#include namespace ProfileEvents diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 75a799e5af1..45a0f33d666 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index f30058fadb9..0b7797da24f 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include namespace DB @@ -29,6 +29,25 @@ void finalizeChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } +void finalizeBlock(Block & block) +{ + for (size_t i = 0; i < block.columns(); ++i) + { + ColumnWithTypeAndName & current = block.getByPosition(i); + const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); + + if (unfinalized_type) + { + current.type = unfinalized_type->getReturnType(); + if (current.column) + { + auto mut_column = IColumn::mutate(std::move(current.column)); + current.column = ColumnAggregateFunction::convertToValues(std::move(mut_column)); + } + } + } +} + Block TotalsHavingTransform::transformHeader( Block block, const ActionsDAG * expression, diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index a1cce03a1a5..57cfa6c01b8 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/DataStreams/BlockIO.cpp b/src/QueryPipeline/BlockIO.cpp similarity index 97% rename from src/DataStreams/BlockIO.cpp rename to src/QueryPipeline/BlockIO.cpp index 692b69388ea..671ba6e4c39 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/QueryPipeline/BlockIO.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/DataStreams/BlockIO.h b/src/QueryPipeline/BlockIO.h similarity index 100% rename from src/DataStreams/BlockIO.h rename to src/QueryPipeline/BlockIO.h diff --git a/src/DataStreams/CMakeLists.txt b/src/QueryPipeline/CMakeLists.txt similarity index 100% rename from src/DataStreams/CMakeLists.txt rename to src/QueryPipeline/CMakeLists.txt diff --git a/src/DataStreams/ConnectionCollector.cpp b/src/QueryPipeline/ConnectionCollector.cpp similarity index 98% rename from src/DataStreams/ConnectionCollector.cpp rename to src/QueryPipeline/ConnectionCollector.cpp index 8e700c0ab7f..65b030fd623 100644 --- a/src/DataStreams/ConnectionCollector.cpp +++ b/src/QueryPipeline/ConnectionCollector.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/ConnectionCollector.h b/src/QueryPipeline/ConnectionCollector.h similarity index 100% rename from src/DataStreams/ConnectionCollector.h rename to src/QueryPipeline/ConnectionCollector.h diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/QueryPipeline/ExecutionSpeedLimits.cpp similarity index 99% rename from src/DataStreams/ExecutionSpeedLimits.cpp rename to src/QueryPipeline/ExecutionSpeedLimits.cpp index e340ee71ab0..2738903cedb 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/QueryPipeline/ExecutionSpeedLimits.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/ExecutionSpeedLimits.h b/src/QueryPipeline/ExecutionSpeedLimits.h similarity index 95% rename from src/DataStreams/ExecutionSpeedLimits.h rename to src/QueryPipeline/ExecutionSpeedLimits.h index b8c320bb005..63658462c9f 100644 --- a/src/DataStreams/ExecutionSpeedLimits.h +++ b/src/QueryPipeline/ExecutionSpeedLimits.h @@ -2,7 +2,7 @@ #include #include -#include +#include class Stopwatch; diff --git a/src/DataStreams/BlockStreamProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp similarity index 67% rename from src/DataStreams/BlockStreamProfileInfo.cpp rename to src/QueryPipeline/ProfileInfo.cpp index 9a06d905223..32986f7259c 100644 --- a/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -8,7 +8,7 @@ namespace DB { -void BlockStreamProfileInfo::read(ReadBuffer & in) +void ProfileInfo::read(ReadBuffer & in) { readVarUInt(rows, in); readVarUInt(blocks, in); @@ -19,7 +19,7 @@ void BlockStreamProfileInfo::read(ReadBuffer & in) } -void BlockStreamProfileInfo::write(WriteBuffer & out) const +void ProfileInfo::write(WriteBuffer & out) const { writeVarUInt(rows, out); writeVarUInt(blocks, out); @@ -30,7 +30,7 @@ void BlockStreamProfileInfo::write(WriteBuffer & out) const } -void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool skip_block_size_info) +void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) { if (!skip_block_size_info) { @@ -44,24 +44,24 @@ void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool sk } -size_t BlockStreamProfileInfo::getRowsBeforeLimit() const +size_t ProfileInfo::getRowsBeforeLimit() const { return rows_before_limit; } -bool BlockStreamProfileInfo::hasAppliedLimit() const +bool ProfileInfo::hasAppliedLimit() const { return applied_limit; } -void BlockStreamProfileInfo::update(Block & block) +void ProfileInfo::update(Block & block) { update(block.rows(), block.bytes()); } -void BlockStreamProfileInfo::update(size_t num_rows, size_t num_bytes) +void ProfileInfo::update(size_t num_rows, size_t num_bytes) { ++blocks; rows += num_rows; diff --git a/src/DataStreams/BlockStreamProfileInfo.h b/src/QueryPipeline/ProfileInfo.h similarity index 90% rename from src/DataStreams/BlockStreamProfileInfo.h rename to src/QueryPipeline/ProfileInfo.h index 1707b941445..335092ce244 100644 --- a/src/DataStreams/BlockStreamProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -13,7 +13,7 @@ class ReadBuffer; class WriteBuffer; /// Information for profiling. See IBlockInputStream.h -struct BlockStreamProfileInfo +struct ProfileInfo { bool started = false; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Time with waiting time @@ -22,7 +22,7 @@ struct BlockStreamProfileInfo size_t blocks = 0; size_t bytes = 0; - using BlockStreamProfileInfos = std::vector; + using ProfileInfos = std::vector; /** Get the number of rows if there were no LIMIT. * If there is no LIMIT, 0 is returned. @@ -42,7 +42,7 @@ struct BlockStreamProfileInfo /// Sets main fields from other object (see methods above). /// If skip_block_size_info if true, then rows, bytes and block fields are ignored. - void setFrom(const BlockStreamProfileInfo & rhs, bool skip_block_size_info); + void setFrom(const ProfileInfo & rhs, bool skip_block_size_info); /// Only for Processors. void setRowsBeforeLimit(size_t rows_before_limit_) diff --git a/src/DataStreams/RemoteBlockOutputStream.cpp b/src/QueryPipeline/RemoteInserter.cpp similarity index 98% rename from src/DataStreams/RemoteBlockOutputStream.cpp rename to src/QueryPipeline/RemoteInserter.cpp index 7642098ff0c..c34c625dc6d 100644 --- a/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/src/QueryPipeline/RemoteInserter.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/DataStreams/RemoteBlockOutputStream.h b/src/QueryPipeline/RemoteInserter.h similarity index 56% rename from src/DataStreams/RemoteBlockOutputStream.h rename to src/QueryPipeline/RemoteInserter.h index f1f49015c9d..0688b555825 100644 --- a/src/DataStreams/RemoteBlockOutputStream.h +++ b/src/QueryPipeline/RemoteInserter.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -44,23 +43,4 @@ private: bool finished = false; }; -class RemoteSink final : public RemoteInserter, public SinkToStorage -{ -public: - explicit RemoteSink( - Connection & connection_, - const ConnectionTimeouts & timeouts, - const String & query_, - const Settings & settings_, - const ClientInfo & client_info_) - : RemoteInserter(connection_, timeouts, query_, settings_, client_info_) - , SinkToStorage(RemoteInserter::getHeader()) - { - } - - String getName() const override { return "RemoteSink"; } - void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } - void onFinish() override { RemoteInserter::onFinish(); } -}; - } diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp similarity index 99% rename from src/DataStreams/RemoteQueryExecutor.cpp rename to src/QueryPipeline/RemoteQueryExecutor.cpp index 3c78fddfd39..23fe3a5f220 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include diff --git a/src/DataStreams/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h similarity index 98% rename from src/DataStreams/RemoteQueryExecutor.h rename to src/QueryPipeline/RemoteQueryExecutor.h index d82f9983894..b7a2509ea97 100644 --- a/src/DataStreams/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -21,8 +21,8 @@ using ThrottlerPtr = std::shared_ptr; struct Progress; using ProgressCallback = std::function; -struct BlockStreamProfileInfo; -using ProfileInfoCallback = std::function; +struct ProfileInfo; +using ProfileInfoCallback = std::function; class RemoteQueryExecutorReadContext; diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp similarity index 99% rename from src/DataStreams/RemoteQueryExecutorReadContext.cpp rename to src/QueryPipeline/RemoteQueryExecutorReadContext.cpp index 6bdf52d2831..5f7b5e24967 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/QueryPipeline/RemoteQueryExecutorReadContext.cpp @@ -1,6 +1,6 @@ #if defined(OS_LINUX) -#include +#include #include #include #include diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/QueryPipeline/RemoteQueryExecutorReadContext.h similarity index 100% rename from src/DataStreams/RemoteQueryExecutorReadContext.h rename to src/QueryPipeline/RemoteQueryExecutorReadContext.h diff --git a/src/DataStreams/SizeLimits.cpp b/src/QueryPipeline/SizeLimits.cpp similarity index 97% rename from src/DataStreams/SizeLimits.cpp rename to src/QueryPipeline/SizeLimits.cpp index 06dde923e55..90005902f67 100644 --- a/src/DataStreams/SizeLimits.cpp +++ b/src/QueryPipeline/SizeLimits.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/DataStreams/SizeLimits.h b/src/QueryPipeline/SizeLimits.h similarity index 100% rename from src/DataStreams/SizeLimits.h rename to src/QueryPipeline/SizeLimits.h diff --git a/src/DataStreams/StreamLocalLimits.h b/src/QueryPipeline/StreamLocalLimits.h similarity index 91% rename from src/DataStreams/StreamLocalLimits.h rename to src/QueryPipeline/StreamLocalLimits.h index efda6a941cc..7f49a5d0b07 100644 --- a/src/DataStreams/StreamLocalLimits.h +++ b/src/QueryPipeline/StreamLocalLimits.h @@ -1,6 +1,6 @@ #pragma once -#include -#include +#include +#include namespace DB { diff --git a/src/DataStreams/examples/CMakeLists.txt b/src/QueryPipeline/examples/CMakeLists.txt similarity index 100% rename from src/DataStreams/examples/CMakeLists.txt rename to src/QueryPipeline/examples/CMakeLists.txt diff --git a/src/DataStreams/narrowBlockInputStreams.cpp b/src/QueryPipeline/narrowBlockInputStreams.cpp similarity index 100% rename from src/DataStreams/narrowBlockInputStreams.cpp rename to src/QueryPipeline/narrowBlockInputStreams.cpp diff --git a/src/DataStreams/narrowBlockInputStreams.h b/src/QueryPipeline/narrowBlockInputStreams.h similarity index 100% rename from src/DataStreams/narrowBlockInputStreams.h rename to src/QueryPipeline/narrowBlockInputStreams.h diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp similarity index 100% rename from src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp rename to src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp diff --git a/src/DataStreams/tests/gtest_check_sorted_stream.cpp b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp similarity index 100% rename from src/DataStreams/tests/gtest_check_sorted_stream.cpp rename to src/QueryPipeline/tests/gtest_check_sorted_stream.cpp diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 5d62a295dad..5b727253dff 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -9,8 +9,7 @@ #include #include #include -#include -#include +#include #include #include #include @@ -595,7 +594,7 @@ namespace void addProgressToResult(); void addTotalsToResult(const Block & totals); void addExtremesToResult(const Block & extremes); - void addProfileInfoToResult(const BlockStreamProfileInfo & info); + void addProfileInfoToResult(const ProfileInfo & info); void addLogsToResult(); void sendResult(); void throwIfFailedToSendResult(); @@ -1381,7 +1380,7 @@ namespace format->doWriteSuffix(); } - void Call::addProfileInfoToResult(const BlockStreamProfileInfo & info) + void Call::addProfileInfoToResult(const ProfileInfo & info) { auto & stats = *result.mutable_stats(); stats.set_rows(info.rows); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f3247e7bc2b..4e8291f5281 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -16,8 +16,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -772,7 +772,7 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() out->next(); } -void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info) +void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); info.write(*out); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index e89d82cfcc8..cb14323906b 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -9,10 +9,10 @@ #include #include #include -#include +#include #include #include -#include +#include #include "IServer.h" @@ -30,7 +30,7 @@ namespace DB class Session; struct Settings; class ColumnsDescription; -struct BlockStreamProfileInfo; +struct ProfileInfo; /// State of query processing. struct QueryState @@ -224,7 +224,7 @@ private: void sendEndOfStream(); void sendPartUUIDs(); void sendReadTaskRequestAssumeLocked(); - void sendProfileInfo(const BlockStreamProfileInfo & info); + void sendProfileInfo(const ProfileInfo & info); void sendTotals(const Block & totals); void sendExtremes(const Block & extremes); diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 167e36ebbe3..254d82520dc 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 1841be22b72..cf349d1f8cf 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -14,8 +14,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/MarkCache.h b/src/Storages/MarkCache.h index ccf8a2e606d..06143e954f8 100644 --- a/src/Storages/MarkCache.h +++ b/src/Storages/MarkCache.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace ProfileEvents diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 9e09cd0036e..a1df4a13e6a 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 7624dc303e0..4fec5ce46bc 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -1,7 +1,7 @@ #pragma once -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index da79b917b5a..d128ecb9e27 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index df92b270542..1dd75f09b92 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -4,7 +4,7 @@ #include -#include +#include #include #include diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 0f47f654428..21784952c23 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -3,7 +3,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageExecutable.h b/src/Storages/StorageExecutable.h index 0d17e003ce4..74df17f1463 100644 --- a/src/Storages/StorageExecutable.h +++ b/src/Storages/StorageExecutable.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 6492c9e07c0..38db0b61e8d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 3bdf3218b2e..a1724af38cb 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index d95a9465bd6..c94c519c3b9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 0e7faad194e..8f7654821cb 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -23,11 +23,11 @@ #include #include #include -#include +#include #include #include "Processors/Sources/SourceWithProgress.h" #include -#include +#include #include #include #include diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index d7eef35e60b..4e2c6cfbe10 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -3,7 +3,7 @@ #if USE_SQLITE #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 2547af1b0ad..77231ce49fd 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -4,9 +4,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include #include #include #include @@ -215,7 +215,7 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) CompressedReadBuffer compressed_backup_buf(*backup_buf); NativeReader backup_stream(compressed_backup_buf, 0); - BlockStreamProfileInfo info; + ProfileInfo info; while (Block block = backup_stream.read()) { info.update(block); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 0cd07afc26c..66f49761793 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -14,8 +14,8 @@ #include #include -#include -#include +#include +#include #include diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 639692beda5..532abb8e2f3 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 005a689f895..80f108eb68a 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 7bd8ad2e740..7e28decfdb0 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include #include #include From da45d55e22b141eb3764ef9eb9812ecaf258e641 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 23:18:37 +0300 Subject: [PATCH 724/950] Remove DataStreams folder. --- utils/wal-dump/main.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/wal-dump/main.cpp b/utils/wal-dump/main.cpp index 0e47c39fb5a..3566936324b 100644 --- a/utils/wal-dump/main.cpp +++ b/utils/wal-dump/main.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include From 6f682d54b2ee0a27465dcebf23c43f4ee1934276 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:21:18 +0300 Subject: [PATCH 725/950] Fixed test --- src/Dictionaries/HashedDictionary.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index eec7cec0285..917eb4cbde3 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -664,10 +664,7 @@ Pipe HashedDictionary::read(const Names & column_na }); } - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); - else - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } template From f3cbac79d343c0fa4e697fbbd40bc7ca0346eeb2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Oct 2021 23:22:24 +0300 Subject: [PATCH 726/950] Try fix integration test. --- src/DataStreams/BlockStreamProfileInfo.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/DataStreams/BlockStreamProfileInfo.cpp b/src/DataStreams/BlockStreamProfileInfo.cpp index 9a06d905223..05c1ac96db0 100644 --- a/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/src/DataStreams/BlockStreamProfileInfo.cpp @@ -46,12 +46,14 @@ void BlockStreamProfileInfo::setFrom(const BlockStreamProfileInfo & rhs, bool sk size_t BlockStreamProfileInfo::getRowsBeforeLimit() const { + calculated_rows_before_limit = true; return rows_before_limit; } bool BlockStreamProfileInfo::hasAppliedLimit() const { + calculated_rows_before_limit = true; return applied_limit; } From a35d7096d8117d1182dcf0d4951d3ab781fbc84d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 15 Oct 2021 23:26:09 +0300 Subject: [PATCH 727/950] Added concept HasIndexOperator --- src/Functions/FunctionsJSON.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 9558b856511..cb55ba6b83b 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -58,12 +58,11 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template -struct HasIndexOperator : std::false_type {}; - -template -struct HasIndexOperator()[0])>> : std::true_type {}; - +template +concept HasIndexOperator = requires (T t) +{ + t[0]; +}; /// Functions to parse JSONs and extract values from it. /// The first argument of all these functions gets a JSON, @@ -285,7 +284,7 @@ private: return true; } - if constexpr (HasIndexOperator::value) + if constexpr (HasIndexOperator) { if (element.isObject()) { From 41acc524581782d54c6985915c5381f1badaef18 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 00:12:51 +0300 Subject: [PATCH 728/950] Preparation to build with Musl --- base/base/LineReader.cpp | 4 +++ .../include/jemalloc/jemalloc_defs.h | 6 +++-- .../include/jemalloc/jemalloc_protos.h | 2 +- .../internal/jemalloc_internal_defs.h.in | 14 +++++----- contrib/libcxx-cmake/CMakeLists.txt | 4 +++ .../linux_x86_64/include/portable.h | 4 ++- src/Common/QueryProfiler.cpp | 8 +++--- src/Common/malloc.cpp | 4 +++ src/IO/BitHelpers.h | 26 +++++-------------- 9 files changed, 40 insertions(+), 32 deletions(-) diff --git a/base/base/LineReader.cpp b/base/base/LineReader.cpp index 8600f4c7b65..d325154ee61 100644 --- a/base/base/LineReader.cpp +++ b/base/base/LineReader.cpp @@ -5,6 +5,10 @@ #include #include +#include +#include +#include + #ifdef OS_LINUX /// We can detect if code is linked with one or another readline variants or open the library dynamically. diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h index 6a03a231a0e..0aa4033f859 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h @@ -18,8 +18,10 @@ * Define overrides for non-standard allocator-related functions if they are * present on the system. */ -#define JEMALLOC_OVERRIDE_MEMALIGN -#define JEMALLOC_OVERRIDE_VALLOC +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE_MEMALIGN + #define JEMALLOC_OVERRIDE_VALLOC +#endif /* * At least Linux omits the "const" in: diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h index 8506237729d..2e35e7b6249 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h @@ -1,6 +1,6 @@ // OSX does not have this for system alloc functions, so you will get // "exception specification in declaration" error. -#if defined(__APPLE__) || defined(__FreeBSD__) +#if defined(__APPLE__) || defined(__FreeBSD__) || defined(USE_MUSL) # undef JEMALLOC_NOTHROW # define JEMALLOC_NOTHROW diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index d5cf0e719ef..44ff2d9fad1 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -13,12 +13,14 @@ * Define overrides for non-standard allocator-related functions if they are * present on the system. */ -#define JEMALLOC_OVERRIDE___LIBC_CALLOC -#define JEMALLOC_OVERRIDE___LIBC_FREE -#define JEMALLOC_OVERRIDE___LIBC_MALLOC -#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN -#define JEMALLOC_OVERRIDE___LIBC_REALLOC -#define JEMALLOC_OVERRIDE___LIBC_VALLOC +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif /* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ /* diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index 0cfb4191619..ac67f2563a3 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -56,6 +56,10 @@ if (USE_UNWIND) target_compile_definitions(cxx PUBLIC -DSTD_EXCEPTION_HAS_STACK_TRACE=1) endif () +if (USE_MUSL) + target_compile_definitions(cxx PUBLIC -D_LIBCPP_HAS_MUSL_LIBC=1) +endif () + # Override the deduced attribute support that causes error. if (OS_DARWIN AND COMPILER_GCC) add_compile_definitions(_LIBCPP_INIT_PRIORITY_MAX) diff --git a/contrib/openldap-cmake/linux_x86_64/include/portable.h b/contrib/openldap-cmake/linux_x86_64/include/portable.h index 2924b6713a4..ab7052bda91 100644 --- a/contrib/openldap-cmake/linux_x86_64/include/portable.h +++ b/contrib/openldap-cmake/linux_x86_64/include/portable.h @@ -98,7 +98,9 @@ #define HAVE_BCOPY 1 /* Define to 1 if you have the header file. */ -#define HAVE_BITS_TYPES_H 1 +#if !defined(USE_MUSL) + #define HAVE_BITS_TYPES_H 1 +#endif /* Define to 1 if you have the `chroot' function. */ #define HAVE_CHROOT 1 diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 7b905937e11..aa40226093a 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -124,11 +124,13 @@ QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const sev.sigev_notify = SIGEV_THREAD_ID; sev.sigev_signo = pause_signal; -# if defined(OS_FREEBSD) +#if defined(OS_FREEBSD) sev._sigev_un._threadid = thread_id; -# else +#elif defined(USE_MUSL) + sev.sigev_notify_thread_id = thread_id; +#else sev._sigev_un._tid = thread_id; -# endif +#endif if (timer_create(clock_type, &sev, &timer_id)) { /// In Google Cloud Run, the function "timer_create" is implemented incorrectly as of 2020-01-25. diff --git a/src/Common/malloc.cpp b/src/Common/malloc.cpp index ec472d5d1d6..88281d9c80e 100644 --- a/src/Common/malloc.cpp +++ b/src/Common/malloc.cpp @@ -17,7 +17,9 @@ extern "C" void *aligned_alloc(size_t alignment, size_t size); void *valloc(size_t size); void *memalign(size_t alignment, size_t size); +#if !defined(USE_MUSL) void *pvalloc(size_t size); +#endif } #pragma GCC diagnostic pop @@ -39,6 +41,8 @@ static void dummyFunctionForInterposing() ignore(aligned_alloc(0, 0)); // -V575 NOLINT ignore(valloc(0)); // -V575 NOLINT ignore(memalign(0, 0)); // -V575 NOLINT +#if !defined(USE_MUSL) ignore(pvalloc(0)); // -V575 NOLINT +#endif } #endif diff --git a/src/IO/BitHelpers.h b/src/IO/BitHelpers.h index bcc36305021..d15297637a3 100644 --- a/src/IO/BitHelpers.h +++ b/src/IO/BitHelpers.h @@ -7,17 +7,6 @@ #include #include -#if defined(__OpenBSD__) || defined(__FreeBSD__) || defined (__ANDROID__) -# include -#elif defined(__sun) -# include -#elif defined(__APPLE__) -# include - -# define htobe64(x) OSSwapHostToBigInt64(x) -# define be64toh(x) OSSwapBigToHostInt64(x) -#endif - namespace DB { @@ -152,7 +141,7 @@ private: memcpy(&tmp_buffer, source_current, bytes_to_read); source_current += bytes_to_read; - tmp_buffer = be64toh(tmp_buffer); + tmp_buffer = __builtin_bswap64(tmp_buffer); bits_buffer |= BufferType(tmp_buffer) << ((sizeof(BufferType) - sizeof(tmp_buffer)) * 8 - bits_count); bits_count += static_cast(bytes_to_read) * 8; @@ -200,7 +189,7 @@ public: capacity = BIT_BUFFER_SIZE - bits_count; } -// write low bits of value as high bits of bits_buffer + // write low bits of value as high bits of bits_buffer const UInt64 mask = maskLowBits(bits_to_write); BufferType v = value & mask; v <<= capacity - bits_to_write; @@ -212,7 +201,7 @@ public: // flush contents of bits_buffer to the dest_current, partial bytes are completed with zeroes. inline void flush() { - bits_count = (bits_count + 8 - 1) & ~(8 - 1); // align UP to 8-bytes, so doFlush will write ALL data from bits_buffer + bits_count = (bits_count + 8 - 1) & ~(8 - 1); // align up to 8-bytes, so doFlush will write all data from bits_buffer while (bits_count != 0) doFlush(); } @@ -231,13 +220,12 @@ private: if (available < to_write) { - throw Exception("Can not write past end of buffer. Space available " - + std::to_string(available) + " bytes, required to write: " - + std::to_string(to_write) + ".", - ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER); + throw Exception(ErrorCodes::CANNOT_WRITE_AFTER_END_OF_BUFFER, + "Can not write past end of buffer. Space available {} bytes, required to write {} bytes.", + available, to_write); } - const auto tmp_buffer = htobe64(static_cast(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8)); + const auto tmp_buffer = __builtin_bswap64(static_cast(bits_buffer >> (sizeof(bits_buffer) - sizeof(UInt64)) * 8)); memcpy(dest_current, &tmp_buffer, to_write); dest_current += to_write; From 0b3bf43d619ff4e753422a8cb0fbdff0d4815662 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 10 Sep 2021 23:56:24 +0300 Subject: [PATCH 729/950] Rewrite MergeTreeData::clearOldTemporaryDirectories() to use early continue --- src/Storages/MergeTree/MergeTreeData.cpp | 47 +++++++++++++----------- 1 file changed, 25 insertions(+), 22 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 790b95a9fa9..37e20204813 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1209,35 +1209,38 @@ void MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifet { for (auto it = disk->iterateDirectory(path); it->isValid(); it->next()) { - if (startsWith(it->name(), "tmp_")) + const std::string & basename = it->name(); + if (!startsWith(basename, "tmp_")) { - try + continue; + } + + try + { + if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) { - if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) - { - LOG_WARNING(log, "Removing temporary directory {}", fullPath(disk, it->path())); - disk->removeRecursive(it->path()); - } + LOG_WARNING(log, "Removing temporary directory {}", fullPath(disk, it->path())); + disk->removeRecursive(it->path()); } - /// see getModificationTime() - catch (const ErrnoException & e) + } + /// see getModificationTime() + catch (const ErrnoException & e) + { + if (e.getErrno() == ENOENT) { - if (e.getErrno() == ENOENT) - { - /// If the file is already deleted, do nothing. - } - else - throw; + /// If the file is already deleted, do nothing. } - catch (const fs::filesystem_error & e) + else + throw; + } + catch (const fs::filesystem_error & e) + { + if (e.code() == std::errc::no_such_file_or_directory) { - if (e.code() == std::errc::no_such_file_or_directory) - { - /// If the file is already deleted, do nothing. - } - else - throw; + /// If the file is already deleted, do nothing. } + else + throw; } } } From 07e8b2b3c7bfe6181287607629002e8aa6bd354c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 11 Sep 2021 00:16:09 +0300 Subject: [PATCH 730/950] Do not try to remove temporary paths that is currently in written by merge/mutation v2: rebase against MergeTask v3: rebase due to conflicts in src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp v4: - rebase due to conflicts in src/Storages/MergeTree/MergeTask.cpp - drop common/scope_guard_safe.h (not used) --- src/Storages/MergeTree/MergeTask.cpp | 20 ++++++++++++++--- src/Storages/MergeTree/MergeTask.h | 3 +++ src/Storages/MergeTree/MergeTreeData.cpp | 10 +++++++-- src/Storages/MergeTree/MergeTreeData.h | 3 ++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 ++++++ .../MergeTree/MergeTreeDataMergerMutator.h | 22 +++++++++++++++++++ .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 9 files changed, 63 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index aa3f91a4f00..2e123d849db 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -10,6 +10,7 @@ #include "Storages/MergeTree/IMergeTreeDataPart.h" #include "Storages/MergeTree/MergeTreeSequentialSource.h" #include "Storages/MergeTree/FutureMergedMutatedPart.h" +#include "Storages/MergeTree/MergeTreeDataMergerMutator.h" #include "Processors/Transforms/ExpressionTransform.h" #include "Processors/Transforms/MaterializingTransform.h" #include "Processors/Merges/MergingSortedTransform.h" @@ -117,11 +118,23 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } ctx->disk = global_ctx->space_reservation->getDisk(); - auto local_new_part_relative_tmp_path_name = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix; - auto local_new_part_tmp_path = global_ctx->data->relative_data_path + local_new_part_relative_tmp_path_name + "/"; + + String local_part_path = global_ctx->data->relative_data_path; + String local_tmp_part_basename = local_tmp_prefix + global_ctx->future_part->name + (global_ctx->parent_part ? ".proj" : ""); + String local_new_part_tmp_path = local_part_path + local_tmp_part_basename + "/"; + if (ctx->disk->exists(local_new_part_tmp_path)) throw Exception("Directory " + fullPath(ctx->disk, local_new_part_tmp_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); + { + std::lock_guard lock(global_ctx->mutator->tmp_parts_lock); + global_ctx->mutator->tmp_parts.emplace(local_tmp_part_basename); + } + SCOPE_EXIT( + std::lock_guard lock(global_ctx->mutator->tmp_parts_lock); + global_ctx->mutator->tmp_parts.erase(local_tmp_part_basename); + ); + global_ctx->all_column_names = global_ctx->metadata_snapshot->getColumns().getNamesOfPhysical(); global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical(); @@ -142,7 +155,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->future_part->type, global_ctx->future_part->part_info, local_single_disk_volume, - local_new_part_relative_tmp_path_name, + local_tmp_part_basename, global_ctx->parent_part); global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; @@ -561,6 +574,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->new_data_part.get(), ".proj", global_ctx->data, + global_ctx->mutator, global_ctx->merges_blocker, global_ctx->ttl_merges_blocker)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 22dc70bd78c..80c8e7165f8 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -60,6 +60,7 @@ public: const IMergeTreeDataPart * parent_part_, String suffix_, MergeTreeData * data_, + MergeTreeDataMergerMutator * mutator_, ActionBlocker * merges_blocker_, ActionBlocker * ttl_merges_blocker_) { @@ -78,6 +79,7 @@ public: global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_); global_ctx->parent_part = std::move(parent_part_); global_ctx->data = std::move(data_); + global_ctx->mutator = std::move(mutator_); global_ctx->merges_blocker = std::move(merges_blocker_); global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_); @@ -121,6 +123,7 @@ private: std::unique_ptr projection_merge_list_element; MergeListElement * merge_list_element_ptr{nullptr}; MergeTreeData * data{nullptr}; + MergeTreeDataMergerMutator * mutator{nullptr}; ActionBlocker * merges_blocker{nullptr}; ActionBlocker * ttl_merges_blocker{nullptr}; StorageMetadataPtr metadata_snapshot{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 37e20204813..83714b814a0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1193,7 +1193,7 @@ static bool isOldPartDirectory(const DiskPtr & disk, const String & directory_pa } -void MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds) +void MergeTreeData::clearOldTemporaryDirectories(const MergeTreeDataMergerMutator & merger_mutator, size_t custom_directories_lifetime_seconds) { /// If the method is already called from another thread, then we don't need to do anything. std::unique_lock lock(clear_old_temporary_directories_mutex, std::defer_lock); @@ -1214,12 +1214,18 @@ void MergeTreeData::clearOldTemporaryDirectories(size_t custom_directories_lifet { continue; } + const std::string & full_path = fullPath(disk, it->path()); + if (merger_mutator.hasTemporaryPart(basename)) + { + LOG_WARNING(log, "{} is an active destination for one of merge/mutation (consider increasing temporary_directories_lifetime setting)", full_path); + continue; + } try { if (disk->isDirectory(it->path()) && isOldPartDirectory(disk, it->path(), deadline)) { - LOG_WARNING(log, "Removing temporary directory {}", fullPath(disk, it->path())); + LOG_WARNING(log, "Removing temporary directory {}", full_path); disk->removeRecursive(it->path()); } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e7f1db8f3ec..a05a106386c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -39,6 +39,7 @@ namespace DB class AlterCommands; class MergeTreePartsMover; +class MergeTreeDataMergerMutator; class MutationCommands; class Context; struct JobAndPool; @@ -536,7 +537,7 @@ public: /// Delete all directories which names begin with "tmp" /// Must be called with locked lockForShare() because it's using relative_data_path. - void clearOldTemporaryDirectories(size_t custom_directories_lifetime_seconds); + void clearOldTemporaryDirectories(const MergeTreeDataMergerMutator & merger_mutator, size_t custom_directories_lifetime_seconds); void clearEmptyParts(); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 5d97c64b49b..f4c23293bf2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -444,6 +444,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( parent_part, suffix, &data, + this, &merges_blocker, &ttl_merges_blocker); } @@ -774,4 +775,10 @@ ExecuteTTLType MergeTreeDataMergerMutator::shouldExecuteTTL(const StorageMetadat } +bool MergeTreeDataMergerMutator::hasTemporaryPart(const std::string & basename) const +{ + std::lock_guard lock(tmp_parts_lock); + return tmp_parts.contains(basename); +} + } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 22650ac4eca..e5c8a4d8285 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -136,6 +137,7 @@ private: MergeTreeData::DataPartsVector selectAllPartsFromPartition(const String & partition_id); friend class MutateTask; + friend class MergeTask; /** Split mutation commands into two parts: * First part should be executed by mutations interpreter. @@ -190,6 +192,26 @@ private: ITTLMergeSelector::PartitionIdToTTLs next_recompress_ttl_merge_times_by_partition; /// Performing TTL merges independently for each partition guarantees that /// there is only a limited number of TTL merges and no partition stores data, that is too stale + +public: + /// Returns true if passed part name is active. + /// (is the destination for one of active mutation/merge). + /// + /// NOTE: that it accept basename (i.e. dirname), not the path, + /// since later requires canonical form. + bool hasTemporaryPart(const std::string & basename) const; + +private: + /// Set of active temporary paths that is used as the destination. + /// List of such paths is required to avoid trying to remove them during cleanup. + /// + /// NOTE: It is pretty short, so use STL is fine. + std::unordered_set tmp_parts; + /// Lock for "tmp_parts". + /// + /// NOTE: mutable is required to mark hasTemporaryPath() const + mutable std::mutex tmp_parts_lock; + }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 06856c73888..5731092f2a8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -62,7 +62,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() /// Both use relative_data_path which changes during rename, so we /// do it under share lock storage.clearOldWriteAheadLogs(); - storage.clearOldTemporaryDirectories(storage.getSettings()->temporary_directories_lifetime.totalSeconds()); + storage.clearOldTemporaryDirectories(storage.merger_mutator, storage.getSettings()->temporary_directories_lifetime.totalSeconds()); } /// This is loose condition: no problem if we actually had lost leadership at this moment diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ab42da1dfa0..0ed906b10f8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -105,7 +105,7 @@ void StorageMergeTree::startup() /// Temporary directories contain incomplete results of merges (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately - clearOldTemporaryDirectories(0); + clearOldTemporaryDirectories(merger_mutator, 0); /// NOTE background task will also do the above cleanups periodically. time_after_previous_cleanup_parts.restart(); @@ -1063,7 +1063,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign assignee.scheduleCommonTask(ExecutableLambdaAdapter::create( [this, share_lock] () { - clearOldTemporaryDirectories(getSettings()->temporary_directories_lifetime.totalSeconds()); + clearOldTemporaryDirectories(merger_mutator, getSettings()->temporary_directories_lifetime.totalSeconds()); return true; }, common_assignee_trigger, getStorageID())); scheduled = true; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..9eb5ab7f800 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -479,7 +479,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( } /// Temporary directories contain uninitialized results of Merges or Fetches (after forced restart), /// don't allow to reinitialize them, delete each of them immediately. - clearOldTemporaryDirectories(0); + clearOldTemporaryDirectories(merger_mutator, 0); clearOldWriteAheadLogs(); } From 55116ae399fdd09bc1808cdeabdf4d665831a697 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 01:57:22 +0300 Subject: [PATCH 731/950] Fix error --- base/base/phdr_cache.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/phdr_cache.cpp b/base/base/phdr_cache.cpp index d2388666f73..8ae10f6bf83 100644 --- a/base/base/phdr_cache.cpp +++ b/base/base/phdr_cache.cpp @@ -6,7 +6,7 @@ #include -#if defined(__linux__) && !defined(THREAD_SANITIZER) +#if defined(__linux__) && !defined(THREAD_SANITIZER) && !defined(USE_MUSL) #define USE_PHDR_CACHE 1 #endif From 4848a6f97fd2203f7b46ea505b51dc395c1b6484 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 02:07:13 +0300 Subject: [PATCH 732/950] Add CMakeLists --- CMakeLists.txt | 6 +----- cmake/linux/default_libs.cmake | 10 +++++++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 685b2c25a0d..2699132f165 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -188,7 +188,7 @@ endif () option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) -if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0") +if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND NOT USE_MUSL) # Only for Linux, x86_64 or aarch64. option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) elseif(GLIBC_COMPATIBILITY) @@ -203,10 +203,6 @@ if (GLIBC_COMPATIBILITY) set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -include ${CMAKE_CURRENT_SOURCE_DIR}/base/glibc-compatibility/glibc-compat-2.32.h") endif() -if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0") - message (WARNING "CMake version must be greater than 3.9.0 for production builds.") -endif () - # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") diff --git a/cmake/linux/default_libs.cmake b/cmake/linux/default_libs.cmake index a2da7ba1915..4abd0a951e1 100644 --- a/cmake/linux/default_libs.cmake +++ b/cmake/linux/default_libs.cmake @@ -14,6 +14,8 @@ endif () if (OS_ANDROID) # pthread and rt are included in libc set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -ldl") +elseif (USE_MUSL) + set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -static -lc") else () set (DEFAULT_LIBS "${DEFAULT_LIBS} ${BUILTINS_LIBRARY} ${COVERAGE_OPTION} -lc -lm -lrt -lpthread -ldl") endif () @@ -26,7 +28,7 @@ set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS}) # glibc-compatibility library relies to constant version of libc headers # (because minor changes in function attributes between different glibc versions will introduce incompatibilities) # This is for x86_64. For other architectures we have separate toolchains. -if (ARCH_AMD64 AND NOT_UNBUNDLED) +if (ARCH_AMD64 AND NOT_UNBUNDLED AND NOT CMAKE_CROSSCOMPILING) set(CMAKE_C_STANDARD_INCLUDE_DIRECTORIES ${ClickHouse_SOURCE_DIR}/contrib/libc-headers/x86_64-linux-gnu ${ClickHouse_SOURCE_DIR}/contrib/libc-headers) set(CMAKE_CXX_STANDARD_INCLUDE_DIRECTORIES ${ClickHouse_SOURCE_DIR}/contrib/libc-headers/x86_64-linux-gnu ${ClickHouse_SOURCE_DIR}/contrib/libc-headers) endif () @@ -37,8 +39,10 @@ set(THREADS_PREFER_PTHREAD_FLAG ON) find_package(Threads REQUIRED) if (NOT OS_ANDROID) - # Our compatibility layer doesn't build under Android, many errors in musl. - add_subdirectory(base/glibc-compatibility) + if (NOT USE_MUSL) + # Our compatibility layer doesn't build under Android, many errors in musl. + add_subdirectory(base/glibc-compatibility) + endif () add_subdirectory(base/harmful) endif () From 2c3709d8ae287d5b3739a1760a0298f8079df050 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 02:07:36 +0300 Subject: [PATCH 733/950] Add toolchain file --- cmake/linux/toolchain-x86_64-musl.cmake | 35 +++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 cmake/linux/toolchain-x86_64-musl.cmake diff --git a/cmake/linux/toolchain-x86_64-musl.cmake b/cmake/linux/toolchain-x86_64-musl.cmake new file mode 100644 index 00000000000..b616a92bb06 --- /dev/null +++ b/cmake/linux/toolchain-x86_64-musl.cmake @@ -0,0 +1,35 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + +set (CMAKE_SYSTEM_NAME "Linux") +set (CMAKE_SYSTEM_PROCESSOR "x86_64") +set (CMAKE_C_COMPILER_TARGET "x86_64-linux-musl") +set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-musl") +set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-musl") + +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl/x86_64-linux-musl-cross") + +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-musl") + +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (USE_MUSL 1) +add_definitions(-DUSE_MUSL=1) From 438d04795e851a29bacfb4f129bed4126aba7275 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 03:03:46 +0300 Subject: [PATCH 734/950] Update toolchain --- cmake/linux/toolchain-x86_64-musl.cmake | 4 ++-- contrib/sysroot | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/linux/toolchain-x86_64-musl.cmake b/cmake/linux/toolchain-x86_64-musl.cmake index b616a92bb06..0406b5de0ba 100644 --- a/cmake/linux/toolchain-x86_64-musl.cmake +++ b/cmake/linux/toolchain-x86_64-musl.cmake @@ -6,9 +6,9 @@ set (CMAKE_C_COMPILER_TARGET "x86_64-linux-musl") set (CMAKE_CXX_COMPILER_TARGET "x86_64-linux-musl") set (CMAKE_ASM_COMPILER_TARGET "x86_64-linux-musl") -set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl/x86_64-linux-musl-cross") +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_64-musl") -set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-musl") +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}") find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") diff --git a/contrib/sysroot b/contrib/sysroot index 002415524b5..e4663925b73 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 002415524b5d14124bb8a61a3ce7ac65774f5479 +Subproject commit e4663925b73beb57dd29154844c8d50441146753 From e7d99c3e544c209537dbc4ee93d65ccada7e22fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Oct 2021 03:09:38 +0300 Subject: [PATCH 735/950] Update submodule --- contrib/fastops | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/fastops b/contrib/fastops index 012b777df9e..1460583af7d 160000 --- a/contrib/fastops +++ b/contrib/fastops @@ -1 +1 @@ -Subproject commit 012b777df9e2d145a24800a6c8c3d4a0249bb09e +Subproject commit 1460583af7d13c0e980ce46aec8ee9400314669a From c18d2834690f1fc9ca5af8727e44823b08cd4631 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Oct 2021 03:57:26 +0300 Subject: [PATCH 736/950] Update codegen_select_fuzzer.cpp --- src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp index 6a7a88a8545..9310d7d59f7 100644 --- a/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp +++ b/src/Parsers/fuzzers/codegen_fuzzer/codegen_select_fuzzer.cpp @@ -25,7 +25,7 @@ DEFINE_BINARY_PROTO_FUZZER(const Sentence& main) std::cout << input << std::endl; DB::ParserQueryWithOutput parser(input.data() + input.size()); - try + try { DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); From c928ce8369ceea17653c0fc5d35fded80b3ddcee Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 16 Oct 2021 04:25:05 +0300 Subject: [PATCH 737/950] improve performance of aggregation in order of primary key --- .../FinishAggregatingInOrderAlgorithm.cpp | 45 ++++++++++++++++--- .../FinishAggregatingInOrderAlgorithm.h | 12 ++++- .../FinishAggregatingInOrderTransform.h | 6 ++- src/Processors/QueryPlan/AggregatingStep.cpp | 3 +- 4 files changed, 54 insertions(+), 12 deletions(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index f488ee61cc3..99e3adf893c 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -25,12 +25,14 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_) + size_t max_block_size_, + size_t merge_threads_) : header(header_) , num_inputs(num_inputs_) , params(params_) , description(std::move(description_)) , max_block_size(max_block_size_) + , pool(merge_threads_) { /// Replace column names in description to positions. for (auto & column_description : description) @@ -58,6 +60,12 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() { + if (finished) + { + auto res = popResult(); + return Status(std::move(res), results.empty()); + } + if (!inputs_to_update.empty()) { Status status(inputs_to_update.back()); @@ -81,7 +89,13 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() } if (!best_input) - return Status{aggregate(), true}; + { + aggregate(); + pool.wait(); + finished = true; + auto res = popResult(); + return Status(std::move(res), results.empty()); + } /// Chunk at best_input will be aggregated entirely. auto & best_state = states[*best_input]; @@ -112,17 +126,34 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() /// Do not merge blocks, if there are too few rows. if (accumulated_rows >= max_block_size) - status.chunk = aggregate(); + aggregate(); + status.chunk = popResult(); return status; } -Chunk FinishAggregatingInOrderAlgorithm::aggregate() +Chunk FinishAggregatingInOrderAlgorithm::popResult() +{ + std::lock_guard lock(results_mutex); + + if (results.empty()) + return {}; + + auto res = std::move(results.back()); + results.pop_back(); + return res; +} + +void FinishAggregatingInOrderAlgorithm::aggregate() { - auto aggregated = params->aggregator.mergeBlocks(blocks, false); - blocks.clear(); accumulated_rows = 0; - return {aggregated.getColumns(), aggregated.rows()}; + pool.scheduleOrThrowOnError([this, blocks_list = std::move(blocks)]() mutable + { + auto aggregated = params->aggregator.mergeBlocks(blocks_list, false); + + std::lock_guard lock(results_mutex); + results.emplace_back(aggregated.getColumns(), aggregated.rows()); + }); } void FinishAggregatingInOrderAlgorithm::addToAggregation() diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 119aefb0ab0..29209f462e4 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -38,15 +39,17 @@ public: size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_); + size_t max_block_size_, + size_t max_threads_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; Status merge() override; private: - Chunk aggregate(); + void aggregate(); void addToAggregation(); + Chunk popResult(); struct State { @@ -69,12 +72,17 @@ private: AggregatingTransformParamsPtr params; SortDescription description; size_t max_block_size; + ThreadPool pool; + + std::mutex results_mutex; + std::vector results; Inputs current_inputs; std::vector states; std::vector inputs_to_update; BlocksList blocks; size_t accumulated_rows = 0; + bool finished = false; }; } diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 6d5e334311f..3a388a36727 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -17,14 +17,16 @@ public: size_t num_inputs, AggregatingTransformParamsPtr params, SortDescription description, - size_t max_block_size) + size_t max_block_size, + size_t merge_threads) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, header, num_inputs, params, std::move(description), - max_block_size) + max_block_size, + merge_threads) { } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 023f9016cc4..4942757cbd8 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -101,7 +101,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B pipeline.getNumStreams(), transform_params, group_by_sort_description, - max_block_size); + max_block_size, + merge_threads); pipeline.addTransform(std::move(transform)); aggregating_sorted = collector.detachProcessors(1); From bf35af1b5d02091f34d2e85387411102140de7c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 16 Oct 2021 09:40:22 +0300 Subject: [PATCH 738/950] Fix ProfileInfo. --- src/QueryPipeline/ProfileInfo.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/QueryPipeline/ProfileInfo.cpp b/src/QueryPipeline/ProfileInfo.cpp index 32986f7259c..ee0ff8c69bf 100644 --- a/src/QueryPipeline/ProfileInfo.cpp +++ b/src/QueryPipeline/ProfileInfo.cpp @@ -46,12 +46,14 @@ void ProfileInfo::setFrom(const ProfileInfo & rhs, bool skip_block_size_info) size_t ProfileInfo::getRowsBeforeLimit() const { + calculated_rows_before_limit = true; return rows_before_limit; } bool ProfileInfo::hasAppliedLimit() const { + calculated_rows_before_limit = true; return applied_limit; } From 6c0eaf76da0a583c8046e7d9c6593f44dc2a8c97 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 11:41:50 +0300 Subject: [PATCH 739/950] Query stage for local --- src/Client/LocalConnection.cpp | 13 ++++++----- src/Client/LocalConnection.h | 2 +- .../02048_clickhouse_local_stage.reference | 15 +++++++++++++ .../02048_clickhouse_local_stage.sh | 22 +++++++++++++++++++ 4 files changed, 45 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02048_clickhouse_local_stage.reference create mode 100755 tests/queries/0_stateless/02048_clickhouse_local_stage.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index efd302622dd..4455ba3b9ad 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -60,15 +60,15 @@ void LocalConnection::updateProgress(const Progress & value) void LocalConnection::sendQuery( const ConnectionTimeouts &, - const String & query_, - const String & query_id_, - UInt64, + const String & query, + const String & query_id, + UInt64 stage, const Settings *, const ClientInfo *, bool) { query_context = session.makeQueryContext(); - query_context->setCurrentQueryId(query_id_); + query_context->setCurrentQueryId(query_id); if (send_progress) query_context->setProgressCallback([this] (const Progress & value) { return this->updateProgress(value); }); @@ -77,8 +77,9 @@ void LocalConnection::sendQuery( state.reset(); state.emplace(); - state->query_id = query_id_; - state->query = query_; + state->query_id = query_id; + state->query = query; + state->stage = QueryProcessingStage::Enum(stage); if (send_progress) state->after_send_progress.restart(); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index dcea3ed0fc3..242d23ddc36 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -76,7 +76,7 @@ public: void sendQuery( const ConnectionTimeouts & timeouts, const String & query, - const String & query_id_/* = "" */, + const String & query_id/* = "" */, UInt64 stage/* = QueryProcessingStage::Complete */, const Settings * settings/* = nullptr */, const ClientInfo * client_info/* = nullptr */, diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.reference b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference new file mode 100644 index 00000000000..44c39f2a444 --- /dev/null +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.reference @@ -0,0 +1,15 @@ +execute: default +"foo" +1 +execute: --stage fetch_columns +"dummy" +0 +execute: --stage with_mergeable_state +"1" +1 +execute: --stage with_mergeable_state_after_aggregation +"1" +1 +execute: --stage complete +"foo" +1 diff --git a/tests/queries/0_stateless/02048_clickhouse_local_stage.sh b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh new file mode 100755 index 00000000000..5c1303b5160 --- /dev/null +++ b/tests/queries/0_stateless/02048_clickhouse_local_stage.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function execute_query() +{ + if [ $# -eq 0 ]; then + echo "execute: default" + else + echo "execute: $*" + fi + ${CLICKHOUSE_LOCAL} "$@" --format CSVWithNames -q "SELECT 1 AS foo" +} + +execute_query # default -- complete +execute_query --stage fetch_columns +execute_query --stage with_mergeable_state +execute_query --stage with_mergeable_state_after_aggregation +execute_query --stage complete From 50231460af503e6e723155fa893e21de1b36f7e0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Oct 2021 11:28:10 +0300 Subject: [PATCH 740/950] Use forward declaration for Buffer<> in generic headers - changes in ReadHelpers.h -- recompiles 1000 modules - changes in FormatFactor.h -- recompiles 100 modules --- src/Dictionaries/SSDCacheDictionaryStorage.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatFactory.h | 7 +++++-- src/Formats/JSONEachRowUtils.cpp | 1 + src/Formats/JSONEachRowUtils.h | 4 ++++ src/IO/ReadHelpers.cpp | 5 +++-- src/IO/ReadHelpers.h | 9 ++++++--- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 1 + .../Formats/Impl/TabSeparatedRowInputFormat.cpp | 1 + 9 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Dictionaries/SSDCacheDictionaryStorage.h b/src/Dictionaries/SSDCacheDictionaryStorage.h index ffe0694d8c4..7c53ecc2b2c 100644 --- a/src/Dictionaries/SSDCacheDictionaryStorage.h +++ b/src/Dictionaries/SSDCacheDictionaryStorage.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 34574ca13f8..ec7fa0a9e80 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -13,6 +13,7 @@ #include #include +#include #include namespace DB diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index f20cec56943..d5784219c6a 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include #include -#include #include #include @@ -34,6 +34,9 @@ struct RowOutputFormatParams; using InputFormatPtr = std::shared_ptr; using OutputFormatPtr = std::shared_ptr; +template +struct Memory; + FormatSettings getFormatSettings(ContextPtr context); template @@ -55,7 +58,7 @@ public: */ using FileSegmentationEngine = std::function( ReadBuffer & buf, - DB::Memory<> & memory, + DB::Memory> & memory, size_t min_chunk_bytes)>; /// This callback allows to perform some additional actions after writing a single row. diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index b918825df79..8ef05fa584e 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index 79dd6c6c192..2d2d4ad5531 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -1,5 +1,9 @@ #pragma once +#include +#include +#include + namespace DB { diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ffa050b71c8..5fe0fda88cd 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -1120,7 +1121,7 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) } } -void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) +void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current) { assert(current >= in.position()); assert(current <= in.buffer().end()); @@ -1140,7 +1141,7 @@ void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current) in.position() = current; } -bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) +bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current) { assert(current <= in.buffer().end()); diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index ca6affbf907..880173bfd96 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -29,7 +30,6 @@ #include #include #include -#include #include #include @@ -41,6 +41,9 @@ static constexpr auto DEFAULT_MAX_STRING_SIZE = 1_GiB; namespace DB { +template +struct Memory; + namespace ErrorCodes { extern const int CANNOT_PARSE_DATE; @@ -1290,7 +1293,7 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); /** This function just copies the data from buffer's internal position (in.position()) * to current position (from arguments) into memory. */ -void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); +void saveUpToPosition(ReadBuffer & in, Memory> & memory, char * current); /** This function is negative to eof(). * In fact it returns whether the data was loaded to internal ReadBuffers's buffer or not. @@ -1299,7 +1302,7 @@ void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); * of our buffer and the current cursor in the end of the buffer. When we call eof() it calls next(). * And this function can fill the buffer with new data, so we will lose the data from previous buffer state. */ -bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current); +bool loadAtPosition(ReadBuffer & in, Memory> & memory, char * & current); struct PcgDeserializer diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8ccc04faf35..4beb260b64a 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 1ff52c9f695..c1cf0a904ea 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include From 77a2022cf543226e2116c6ca896f7b226c0f9364 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 9 Oct 2021 17:00:39 +0800 Subject: [PATCH 741/950] Add primary key to minmax_count_projection --- src/Storages/MergeTree/MergeTreeData.cpp | 120 +++++++++++++----- src/Storages/MergeTree/MergeTreeData.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +- .../MergeTree/registerStorageMergeTree.cpp | 18 +-- src/Storages/ProjectionsDescription.cpp | 19 ++- src/Storages/ProjectionsDescription.h | 6 +- .../01710_minmax_count_projection.reference | 5 + .../01710_minmax_count_projection.sql | 28 ++++ 8 files changed, 157 insertions(+), 44 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c5b5e2a9a92..66b7f62a7c8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4398,6 +4398,8 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, const SelectQueryInfo & query_info, + const DataPartsVector & parts, + DataPartsVector & normal_parts, ContextPtr query_context) const { if (!metadata_snapshot->minmax_count_projection) @@ -4406,7 +4408,14 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( ErrorCodes::LOGICAL_ERROR); auto block = metadata_snapshot->minmax_count_projection->sample_block; + String primary_key_max_column_name; + if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + primary_key_max_column_name = *(block.getNames().cend() - 2); + bool need_primary_key_max_column = std::any_of( + required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); + auto minmax_count_columns = block.mutateColumns(); + auto minmax_count_columns_size = minmax_count_columns.size(); auto insert = [](ColumnAggregateFunction & column, const Field & value) { @@ -4422,7 +4431,6 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( column.insertFrom(place); }; - auto parts = getDataPartsVector(); ASTPtr expression_ast; Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, false /* one_part */, true /* ignore_empty */); if (virtual_columns_block.rows() == 0) @@ -4446,15 +4454,13 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( if (!part->minmax_idx->initialized) throw Exception("Found a non-empty part with uninitialized minmax_idx. It's a bug", ErrorCodes::LOGICAL_ERROR); - size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); - if (2 * minmax_idx_size + 1 != minmax_count_columns.size()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "minmax_count projection should have twice plus one the number of ranges in minmax_idx. 2 * minmax_idx_size + 1 = {}, " - "minmax_count_columns.size() = {}. It's a bug", - 2 * minmax_idx_size + 1, - minmax_count_columns.size()); + if (need_primary_key_max_column && !part->index_granularity.hasFinalMark()) + { + normal_parts.push_back(part); + continue; + } + size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); for (size_t i = 0; i < minmax_idx_size; ++i) { size_t min_pos = i * 2; @@ -4466,6 +4472,16 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( insert(max_column, range.right); } + if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + { + const auto & primary_key_column = *part->index[0]; + auto primary_key_column_size = primary_key_column.size(); + auto & min_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 3]); + auto & max_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 2]); + insert(min_column, primary_key_column[0]); + insert(max_column, primary_key_column[primary_key_column_size - 1]); + } + { auto & column = assert_cast(*minmax_count_columns.back()); auto func = column.getAggregateFunction(); @@ -4722,33 +4738,74 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( size_t min_sum_marks = std::numeric_limits::max(); if (metadata_snapshot->minmax_count_projection) add_projection_candidate(*metadata_snapshot->minmax_count_projection); + std::optional minmax_conut_projection_candidate; + if (!candidates.empty()) + { + minmax_conut_projection_candidate.emplace(std::move(candidates.front())); + candidates.clear(); + } + MergeTreeDataSelectExecutor reader(*this); + std::shared_ptr max_added_blocks; + if (settings.select_sequential_consistency) + { + if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) + max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); + } + auto parts = getDataPartsVector(); - // Only add more projection candidates if minmax_count_projection cannot match. - if (candidates.empty()) + // If minmax_count_projection is a valid candidate, check its completeness. + if (minmax_conut_projection_candidate) + { + DataPartsVector normal_parts; + query_info.minmax_count_projection_block = getMinMaxCountProjectionBlock( + metadata_snapshot, minmax_conut_projection_candidate->required_columns, query_info, parts, normal_parts, query_context); + + if (normal_parts.empty()) + { + selected_candidate = &*minmax_conut_projection_candidate; + selected_candidate->complete = true; + min_sum_marks = query_info.minmax_count_projection_block.rows(); + } + else + { + if (normal_parts.size() == parts.size()) + { + // minmax_count_projection is useless. + } + else + { + auto normal_result_ptr = reader.estimateNumMarksToRead( + normal_parts, + analysis_result.required_columns, + metadata_snapshot, + metadata_snapshot, + query_info, + query_context, + settings.max_threads, + max_added_blocks); + + if (!normal_result_ptr->error()) + { + selected_candidate = &*minmax_conut_projection_candidate; + selected_candidate->merge_tree_normal_select_result_ptr = normal_result_ptr; + min_sum_marks = query_info.minmax_count_projection_block.rows() + normal_result_ptr->marks(); + } + } + + // We cannot find a complete match of minmax_count_projection, add more projections to check. + for (const auto & projection : metadata_snapshot->projections) + add_projection_candidate(projection); + } + } + else { for (const auto & projection : metadata_snapshot->projections) add_projection_candidate(projection); } - else - { - selected_candidate = &candidates.front(); - query_info.minmax_count_projection_block - = getMinMaxCountProjectionBlock(metadata_snapshot, selected_candidate->required_columns, query_info, query_context); - min_sum_marks = query_info.minmax_count_projection_block.rows(); - } // Let's select the best projection to execute the query. - if (!candidates.empty() && !selected_candidate) + if (!candidates.empty()) { - std::shared_ptr max_added_blocks; - if (settings.select_sequential_consistency) - { - if (const StorageReplicatedMergeTree * replicated = dynamic_cast(this)) - max_added_blocks = std::make_shared(replicated->getMaxAddedBlocks()); - } - - auto parts = getDataPartsVector(); - MergeTreeDataSelectExecutor reader(*this); query_info.merge_tree_select_result_ptr = reader.estimateNumMarksToRead( parts, analysis_result.required_columns, @@ -4763,7 +4820,12 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( { // Add 1 to base sum_marks so that we prefer projections even when they have equal number of marks to read. // NOTE: It is not clear if we need it. E.g. projections do not support skip index for now. - min_sum_marks = query_info.merge_tree_select_result_ptr->marks() + 1; + auto sum_marks = query_info.merge_tree_select_result_ptr->marks() + 1; + if (sum_marks < min_sum_marks) + { + selected_candidate = nullptr; + min_sum_marks = sum_marks; + } } /// Favor aggregate projections diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index b87a756bf9f..009abbb7b3b 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -368,6 +368,8 @@ public: const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, const SelectQueryInfo & query_info, + const DataPartsVector & parts, + DataPartsVector & normal_parts, ContextPtr query_context) const; bool getQueryProcessingStageWithAggregateProjection( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a33296cbf24..55de92735c3 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -162,7 +162,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( LOG_DEBUG( log, - "Choose {} projection {}", + "Choose {} {} projection {}", + query_info.projection->complete ? "complete" : "incomplete", query_info.projection->desc->type, query_info.projection->desc->name); diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 379573a381f..982acfe62a4 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -651,10 +651,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// single default partition with name "all". metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_key, metadata.columns, args.getContext()); - auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); - metadata.minmax_count_projection.emplace( - ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, args.getContext())); - /// PRIMARY KEY without ORDER BY is allowed and considered as ORDER BY. if (!args.storage_def->order_by && args.storage_def->primary_key) args.storage_def->set(args.storage_def->order_by, args.storage_def->primary_key->clone()); @@ -686,6 +682,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.primary_key.definition_ast = nullptr; } + auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); + auto primary_key_asts = metadata.primary_key.expression_list_ast->children; + metadata.minmax_count_projection.emplace( + ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, primary_key_asts, args.getContext())); + if (args.storage_def->sample_by) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, args.getContext()); @@ -736,10 +737,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.partition_key = KeyDescription::getKeyFromAST(partition_by_ast, metadata.columns, args.getContext()); - auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); - metadata.minmax_count_projection.emplace( - ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, args.getContext())); - ++arg_num; /// If there is an expression for sampling @@ -765,6 +762,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) ++arg_num; + auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); + auto primary_key_asts = metadata.primary_key.expression_list_ast->children; + metadata.minmax_count_projection.emplace( + ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, primary_key_asts, args.getContext())); + const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) storage_settings->index_granularity = safeGet(ast->value); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 42294b8152c..5ad79304e27 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -60,6 +60,7 @@ ProjectionDescription ProjectionDescription::clone() const other.metadata = metadata; other.key_size = key_size; other.is_minmax_count_projection = is_minmax_count_projection; + other.has_primary_key_minmax = has_primary_key_minmax; return other; } @@ -172,9 +173,15 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const return result; } -ProjectionDescription -ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & columns, const Names & minmax_columns, ContextPtr query_context) +ProjectionDescription ProjectionDescription::getMinMaxCountProjection( + const ColumnsDescription & columns, + const Names & minmax_columns, + const ASTs & primary_key_asts, + ContextPtr query_context) { + ProjectionDescription result; + result.is_minmax_count_projection = true; + auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); for (const auto & column : minmax_columns) @@ -182,10 +189,15 @@ ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & colum select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); } + if (!primary_key_asts.empty()) + { + select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); + select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); + result.has_primary_key_minmax = true; + } select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); - ProjectionDescription result; result.definition_ast = select_query; result.name = MINMAX_COUNT_PROJECTION_NAME; result.query_ast = select_query->cloneToASTSelect(); @@ -203,7 +215,6 @@ ProjectionDescription::getMinMaxCountProjection(const ColumnsDescription & colum metadata.sorting_key = KeyDescription::buildEmptyKey(); metadata.primary_key = KeyDescription::buildEmptyKey(); result.metadata = std::make_shared(metadata); - result.is_minmax_count_projection = true; return result; } diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index b9c11cb0771..28cff7ce532 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -58,12 +58,14 @@ struct ProjectionDescription bool is_minmax_count_projection = false; + bool has_primary_key_minmax = false; + /// Parse projection from definition AST static ProjectionDescription getProjectionFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr query_context); - static ProjectionDescription - getMinMaxCountProjection(const ColumnsDescription & columns, const Names & minmax_columns, ContextPtr query_context); + static ProjectionDescription getMinMaxCountProjection( + const ColumnsDescription & columns, const Names & minmax_columns, const ASTs & primary_key_asts, ContextPtr query_context); ProjectionDescription() = default; diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index ad9b87b998d..5591d5a9954 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -3,3 +3,8 @@ 1 9999 5000 0 9998 5000 1 +0 +0 +0 +0 9999 +0 9999 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 58af11f01f7..112487b219e 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -15,3 +15,31 @@ select min(i), max(i), count() from d where _partition_value.1 = 10 group by _pa select min(i) from d where 1 = _partition_value.1; drop table d; + +drop table if exists no_final_mark; +drop table if exists has_final_mark; +drop table if exists mixed_final_mark; + +create table no_final_mark (i int, j int) engine MergeTree partition by i % 2 order by j settings index_granularity = 10, write_final_mark = 0; +create table has_final_mark (i int, j int) engine MergeTree partition by i % 2 order by j settings index_granularity = 10, write_final_mark = 1; +create table mixed_final_mark (i int, j int) engine MergeTree partition by i % 2 order by j settings index_granularity = 10; + +set max_rows_to_read = 100000; + +insert into no_final_mark select number, number from numbers(10000); +insert into has_final_mark select number, number from numbers(10000); + +alter table mixed_final_mark attach partition 0 from no_final_mark; +alter table mixed_final_mark attach partition 1 from has_final_mark; + +set max_rows_to_read = 2; + +select min(j) from no_final_mark; +select min(j) from has_final_mark; +select min(j) from mixed_final_mark; + +select min(j), max(j) from no_final_mark; -- {serverError TOO_MANY_ROWS} +select min(j), max(j) from has_final_mark; + +set max_rows_to_read = 5001; -- one normal part 5000 + one minmax_count_projection part 1 +select min(j), max(j) from mixed_final_mark; From 77c4a5fa18ea84a33d65d2e67b5bf7d5b9c67263 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 14 Oct 2021 01:31:37 +0800 Subject: [PATCH 742/950] Better --- src/Storages/MergeTree/MergeTreeData.cpp | 38 ++++++++++++------------ src/Storages/MergeTree/MergeTreeData.h | 10 +++++++ src/Storages/ProjectionsDescription.cpp | 10 +++---- src/Storages/ProjectionsDescription.h | 5 ++++ 4 files changed, 39 insertions(+), 24 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 66b7f62a7c8..1f1b9a3f0a8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4408,15 +4408,16 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( ErrorCodes::LOGICAL_ERROR); auto block = metadata_snapshot->minmax_count_projection->sample_block; + bool need_primary_key_max_column = false; String primary_key_max_column_name; if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) - primary_key_max_column_name = *(block.getNames().cend() - 2); - bool need_primary_key_max_column = std::any_of( - required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); + { + primary_key_max_column_name = block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; + need_primary_key_max_column = std::any_of( + required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); + } auto minmax_count_columns = block.mutateColumns(); - auto minmax_count_columns_size = minmax_count_columns.size(); - auto insert = [](ColumnAggregateFunction & column, const Field & value) { auto func = column.getAggregateFunction(); @@ -4460,28 +4461,27 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( continue; } - size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); - for (size_t i = 0; i < minmax_idx_size; ++i) - { - size_t min_pos = i * 2; - size_t max_pos = i * 2 + 1; - auto & min_column = assert_cast(*minmax_count_columns[min_pos]); - auto & max_column = assert_cast(*minmax_count_columns[max_pos]); - const auto & range = part->minmax_idx->hyperrectangle[i]; - insert(min_column, range.left); - insert(max_column, range.right); - } - + size_t pos = 0; if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) { const auto & primary_key_column = *part->index[0]; auto primary_key_column_size = primary_key_column.size(); - auto & min_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 3]); - auto & max_column = assert_cast(*minmax_count_columns[minmax_count_columns_size - 2]); + auto & min_column = assert_cast(*minmax_count_columns[pos++]); + auto & max_column = assert_cast(*minmax_count_columns[pos++]); insert(min_column, primary_key_column[0]); insert(max_column, primary_key_column[primary_key_column_size - 1]); } + size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); + for (size_t i = 0; i < minmax_idx_size; ++i) + { + auto & min_column = assert_cast(*minmax_count_columns[pos++]); + auto & max_column = assert_cast(*minmax_count_columns[pos++]); + const auto & range = part->minmax_idx->hyperrectangle[i]; + insert(min_column, range.left); + insert(max_column, range.right); + } + { auto & column = assert_cast(*minmax_count_columns.back()); auto func = column.getAggregateFunction(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 009abbb7b3b..7151141cb15 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -364,6 +364,16 @@ public: bool attach, BrokenPartCallback broken_part_callback_ = [](const String &){}); + /// Build a block of minmax and count values of a MergeTree table. These values are extracted + /// from minmax_indices, the first expression of primary key, and part rows. + /// + /// query_info - used to filter unneeded parts + /// + /// parts - part set to filter + /// + /// normal_parts - collects parts that don't have all the needed values to form the block. + /// Specifically, this is when a part doesn't contain a final mark and the related max value is + /// required. Block getMinMaxCountProjectionBlock( const StorageMetadataPtr & metadata_snapshot, const Names & required_columns, diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 5ad79304e27..7b27604730c 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -184,17 +184,17 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); - for (const auto & column : minmax_columns) - { - select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); - select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); - } if (!primary_key_asts.empty()) { select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); result.has_primary_key_minmax = true; } + for (const auto & column : minmax_columns) + { + select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); + select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); + } select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 28cff7ce532..77b858b3ab1 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -30,6 +30,10 @@ struct ProjectionDescription static constexpr const char * MINMAX_COUNT_PROJECTION_NAME = "_minmax_count_projection"; + /// If minmax_count projection contains a primary key's minmax values. Their positions will be 0 and 1. + static constexpr const size_t PRIMARY_KEY_MIN_COLUMN_POS = 0; + static constexpr const size_t PRIMARY_KEY_MAX_COLUMN_POS = 1; + /// Definition AST of projection ASTPtr definition_ast; @@ -58,6 +62,7 @@ struct ProjectionDescription bool is_minmax_count_projection = false; + /// If a primary key expression is used in the minmax_count projection, this flag will be true. bool has_primary_key_minmax = false; /// Parse projection from definition AST From 9ceb668a4ddb614291ae74bab2b4b582bf6750ec Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 16 Oct 2021 18:13:11 +0800 Subject: [PATCH 743/950] Fix TSan --- src/Storages/MergeTree/MergeTreeData.cpp | 9 ++++----- src/Storages/ProjectionsDescription.cpp | 5 +++-- src/Storages/ProjectionsDescription.h | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1f1b9a3f0a8..45748539e9d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4407,12 +4407,11 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( "Cannot find the definition of minmax_count projection but it's used in current query. It's a bug", ErrorCodes::LOGICAL_ERROR); - auto block = metadata_snapshot->minmax_count_projection->sample_block; + auto block = metadata_snapshot->minmax_count_projection->sample_block.cloneEmpty(); bool need_primary_key_max_column = false; - String primary_key_max_column_name; - if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + const auto & primary_key_max_column_name = metadata_snapshot->minmax_count_projection->primary_key_max_column_name; + if (!primary_key_max_column_name.empty()) { - primary_key_max_column_name = block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; need_primary_key_max_column = std::any_of( required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); } @@ -4462,7 +4461,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } size_t pos = 0; - if (metadata_snapshot->minmax_count_projection->has_primary_key_minmax) + if (!primary_key_max_column_name.empty()) { const auto & primary_key_column = *part->index[0]; auto primary_key_column_size = primary_key_column.size(); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 7b27604730c..b3b344e9ecc 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -60,7 +60,7 @@ ProjectionDescription ProjectionDescription::clone() const other.metadata = metadata; other.key_size = key_size; other.is_minmax_count_projection = is_minmax_count_projection; - other.has_primary_key_minmax = has_primary_key_minmax; + other.primary_key_max_column_name = primary_key_max_column_name; return other; } @@ -188,7 +188,6 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( { select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); - result.has_primary_key_minmax = true; } for (const auto & column : minmax_columns) { @@ -208,6 +207,8 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); + if (!primary_key_asts.empty()) + result.primary_key_max_column_name = result.sample_block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; result.type = ProjectionDescription::Type::Aggregate; StorageInMemoryMetadata metadata; metadata.setColumns(ColumnsDescription(result.sample_block.getNamesAndTypesList())); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 77b858b3ab1..4dd717239ad 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -62,8 +62,8 @@ struct ProjectionDescription bool is_minmax_count_projection = false; - /// If a primary key expression is used in the minmax_count projection, this flag will be true. - bool has_primary_key_minmax = false; + /// If a primary key expression is used in the minmax_count projection, store the name of max expression. + String primary_key_max_column_name; /// Parse projection from definition AST static ProjectionDescription From 9525437499311d154198bc9b8e1e22d95986c600 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 13:17:00 +0300 Subject: [PATCH 744/950] Less threads in local, fix Ok. printing --- programs/local/LocalServer.cpp | 5 +- src/Client/LocalConnection.cpp | 17 +++--- src/Interpreters/Context.cpp | 9 ++++ src/Interpreters/Context.h | 3 ++ src/Interpreters/InterpreterCreateQuery.cpp | 9 ++++ .../02049_clickhouse_local_merge_tree.expect | 53 +++++++++++++++++++ ...2049_clickhouse_local_merge_tree.reference | 0 7 files changed, 84 insertions(+), 12 deletions(-) create mode 100755 tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect create mode 100644 tests/queries/0_stateless/02049_clickhouse_local_merge_tree.reference diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 30082caaac1..cdd5ae13f99 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -514,19 +514,16 @@ void LocalServer::processConfig() format = config().getString("output-format", config().getString("format", is_interactive ? "PrettyCompact" : "TSV")); insert_format = "Values"; + /// Setting value from cmd arg overrides one from config if (global_context->getSettingsRef().max_insert_block_size.changed) insert_format_max_block_size = global_context->getSettingsRef().max_insert_block_size; else insert_format_max_block_size = config().getInt("insert_format_max_block_size", global_context->getSettingsRef().max_insert_block_size); - /// Skip networking - /// Sets external authenticators config (LDAP, Kerberos). global_context->setExternalAuthenticatorsConfig(config()); - global_context->initializeBackgroundExecutors(); - setupUsers(); /// Limit on total number of concurrently executing queries. diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index efd302622dd..e1324146330 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -266,19 +266,19 @@ bool LocalConnection::poll(size_t) } } - if (state->is_finished && send_progress && !state->sent_progress) - { - state->sent_progress = true; - next_packet_type = Protocol::Server::Progress; - return true; - } - if (state->is_finished) { finishQuery(); return true; } + if (send_progress && !state->sent_progress) + { + state->sent_progress = true; + next_packet_type = Protocol::Server::Progress; + return true; + } + if (state->block && state->block.value()) { next_packet_type = Protocol::Server::Data; @@ -292,7 +292,8 @@ bool LocalConnection::pollImpl() { Block block; auto next_read = pullBlock(block); - if (block) + + if (block && !state->io.null_format) { state->block.emplace(block); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 98acc786aa9..0ef92eaed39 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2895,8 +2895,15 @@ void Context::setAsynchronousInsertQueue(const std::shared_ptrasync_insert_queue = ptr; } +bool Context::isBackgroundExecutorsInitialized() const +{ + return is_background_executors_initialized; +} + void Context::initializeBackgroundExecutors() { + assert(!is_background_executors_initialized); + const size_t max_merges_and_mutations = getSettingsRef().background_pool_size * getSettingsRef().background_merges_mutations_concurrency_ratio; /// With this executor we can execute more tasks than threads we have @@ -2943,6 +2950,8 @@ void Context::initializeBackgroundExecutors() LOG_INFO(shared->log, "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}", getSettingsRef().background_common_pool_size, getSettingsRef().background_common_pool_size); + + is_background_executors_initialized = true; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 247dbc74f22..15c4376aa6d 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -293,6 +293,8 @@ private: /// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL). bool is_internal_query = false; + /// Has initializeBackgroundExecutors() method been executed? + bool is_background_executors_initialized = false; public: @@ -862,6 +864,7 @@ public: /// Background executors related methods void initializeBackgroundExecutors(); + bool isBackgroundExecutorsInitialized() const; MergeMutateBackgroundExecutorPtr getMergeMutateExecutor() const; OrdinaryBackgroundExecutorPtr getMovesExecutor() const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6d38c55bd62..5b993bce724 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -833,6 +833,15 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = getContext()->getCurrentDatabase(); auto database_name = create.database.empty() ? current_database : create.database; + auto global_context = getContext()->getGlobalContext(); + if (global_context + && global_context->getApplicationType() == Context::ApplicationType::LOCAL + && !global_context->isBackgroundExecutorsInitialized() + && create.storage && endsWith(create.storage->engine->name, "MergeTree")) + { + global_context->initializeBackgroundExecutors(); + } + // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect new file mode 100755 index 00000000000..17b98b077d5 --- /dev/null +++ b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect @@ -0,0 +1,53 @@ +#!/usr/bin/expect -f +# Tags: no-fasttest + +log_user 0 +set timeout 20 +match_max 100000 + +# A default timeout action is to fail +expect_after { + timeout { + exit 1 + } + +} + +set basedir [file dirname $argv0] +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_LOCAL --disable_suggestion" +expect ":) " + +send -- "drop table if exists t\r" +expect "Ok." + +send -- "create table t engine=MergeTree() order by tuple() as select 1\r" +expect "Ok." + +send -- "set optimize_on_insert = 0\r" +expect "Ok." + +send -- "drop table if exists tt\r" +expect "Ok." + +send -- "create table tt (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date\r" +expect "Ok." + +send -- "insert into tt values ('2020-01-01', 2, 2), ('2020-01-01', 1, 1)\r" +expect "Ok." + +send -- "insert into tt values ('2020-01-01', 0, 0)\r" +expect "Ok." + +send -- "OPTIMIZE TABLE tt\r" +expect "Ok." + +send -- "select * from tt order by version format TSV\r" +expect "2020-01-01\t2\t2" + +send -- "drop table tt\r" +expect "Ok." +send -- "drop table t\r" +expect "Ok." + +send -- "\4" +expect eof diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.reference b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.reference new file mode 100644 index 00000000000..e69de29bb2d From 1b50d26c5122df276379e4cd5a2d86aa456c65de Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 16 Oct 2021 13:41:56 +0300 Subject: [PATCH 745/950] Added documentation --- .../external-dicts-dict-layout.md | 54 +++++++++++++++++++ src/Dictionaries/HashedArrayDictionary.cpp | 11 ++-- 2 files changed, 60 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index f525ea64aa2..30e050ef9ef 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -155,6 +155,60 @@ Configuration example: LAYOUT(COMPLEX_KEY_HASHED()) ``` +### complex_key_sparse_hashed {#complex-key-sparse-hashed} + +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `sparse_hashed`. + +Configuration example: + +``` xml + + + +``` + +``` sql +LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) +``` + +### hashed_array {#dicts-external_dicts_dict_layout-hashed-array} + +The dictionary is completely stored in memory. Each attribute is stored in array. Key attribute is stored in the form of hashed table where value is index in attributes array. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. + +All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. + +Configuration example: + +``` xml + + + + +``` + +or + +``` sql +LAYOUT(HASHED_ARRAY()) +``` + +### complex_key_hashed_array {#complex-key-hashed-array} + +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed_array`. + +Configuration example: + +``` xml + + + +``` + +``` sql +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) +``` + + ### range_hashed {#range-hashed} The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 7706f7d6108..3c02f377c3e 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -622,6 +622,9 @@ void HashedArrayDictionary::calculateBytesAllocated() if (attribute.string_arena) bytes_allocated += attribute.string_arena->size(); + + if (attribute.is_index_null.has_value()) + bytes_allocated += (*attribute.is_index_null).size(); } bytes_allocated += complex_key_arena.size(); @@ -634,14 +637,12 @@ template Pipe HashedArrayDictionary::read(const Names & column_names, size_t max_block_size) const { PaddedPODArray keys; + keys.reserve(key_attribute.container.size()); - for (auto & [key, value] : key_attribute.container) + for (auto & [key, _] : key_attribute.container) keys.emplace_back(key); - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); - else - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } template class HashedArrayDictionary; From 8a94e26bece5c3d6b5206acda686f1b0f7de4229 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sat, 16 Oct 2021 18:51:42 +0800 Subject: [PATCH 746/950] init --- src/Functions/ReplaceRegexpImpl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 3e80dd5b337..1caced9cbde 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -110,7 +110,7 @@ struct ReplaceRegexpImpl res_data.resize(res_data.size() + bytes_to_copy); memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, bytes_to_copy); res_offset += bytes_to_copy; - start_pos += bytes_to_copy + match.length(); + start_pos += bytes_to_copy + (match.length() > 0 ? match.length() : 1); /// Do substitution instructions for (const auto & it : instructions) @@ -129,7 +129,7 @@ struct ReplaceRegexpImpl } } - if (replace_one || match.length() == 0) /// Stop after match of zero length, to avoid infinite loop. + if (replace_one) /// Stop after match of zero length, to avoid infinite loop. can_finish_current_string = true; } else From c5b09b49079581527502b35ecd7efb83d93e47f6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Sat, 16 Oct 2021 13:03:51 +0200 Subject: [PATCH 747/950] Tag resource heavy tests as no-parallel --- .../01509_check_many_parallel_quorum_inserts_long.sh | 2 +- tests/queries/0_stateless/02033_join_engine_deadlock_long.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index 6533eeb12f5..a6f151d0f6f 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database +# Tags: long, no-replicated-database, no-parallel # Tag no-replicated-database: Fails due to additional replicas or shards set -e diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh index 2a887cbbcae..1e670b72fe7 100755 --- a/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh +++ b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, deadlock +# Tags: long, deadlock, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From c79c0526ff5f9783648a85e6c87657ecb9d2d060 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 15:18:00 +0300 Subject: [PATCH 748/950] Fix clickhouse-local syntax exception --- programs/local/LocalServer.cpp | 6 ++---- .../02050_clickhouse_local_parsing_exception.reference | 1 + .../02050_clickhouse_local_parsing_exception.sh | 8 ++++++++ 3 files changed, 11 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference create mode 100755 tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 30082caaac1..10c4bf592ca 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -128,9 +127,8 @@ bool LocalServer::executeMultiQuery(const String & all_queries_text) } case MultiQueryProcessingStage::PARSING_EXCEPTION: { - this_query_end = find_first_symbols<'\n'>(this_query_end, all_queries_end); - this_query_begin = this_query_end; /// It's expected syntax error, skip the line - current_exception.reset(); + if (current_exception) + current_exception->rethrow(); continue; } case MultiQueryProcessingStage::EXECUTE_QUERY: diff --git a/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh new file mode 100755 index 00000000000..7a92fa6fefe --- /dev/null +++ b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --query="SELECT number FROM system.numbers INTO OUTFILE test.native.zst FORMAT Native" 2>&1 | grep -q "Code: 62. DB::Exception: Syntax error: failed at position 48 ('test'): test.native.zst FORMAT Native. Expected string literal." && echo 'OK' || echo 'FAIL' ||: + From 1a947d30bf4f074292f8ca15a066fc599d1b5d90 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 16 Oct 2021 16:41:54 +0300 Subject: [PATCH 749/950] add perf test --- .../performance/read_in_order_many_parts.xml | 26 ++++++++++++------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/tests/performance/read_in_order_many_parts.xml b/tests/performance/read_in_order_many_parts.xml index ca713935426..cc69bfeb163 100644 --- a/tests/performance/read_in_order_many_parts.xml +++ b/tests/performance/read_in_order_many_parts.xml @@ -15,17 +15,25 @@ 1000 + + uniqs + + 100 + 10000 + 1000000 + + - CREATE TABLE mt_{parts}_parts(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % {parts} SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 - SYSTEM STOP MERGES mt_{parts}_parts - INSERT INTO mt_{parts}_parts SELECT number, rand() % 10000, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 - SYSTEM START MERGES mt_{parts}_parts - OPTIMIZE TABLE mt_{parts}_parts FINAL + CREATE TABLE mt_{parts}_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % {parts} SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 + SYSTEM STOP MERGES mt_{parts}_parts_{uniqs}_uniqs + INSERT INTO mt_{parts}_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 + SYSTEM START MERGES mt_{parts}_parts_{uniqs}_uniqs + OPTIMIZE TABLE mt_{parts}_parts_{uniqs}_uniqs FINAL - SELECT val2 FROM mt_{parts}_parts ORDER BY val1 LIMIT 1000000 FORMAT Null - SELECT val2 FROM mt_{parts}_parts ORDER BY val1 LIMIT 10000000 FORMAT Null - SELECT sum(val2) FROM mt_{parts}_parts GROUP BY val1 FORMAT Null + SELECT val2 FROM mt_{parts}_parts_10000_uniqs ORDER BY val1 LIMIT 1000000 FORMAT Null + SELECT val2 FROM mt_{parts}_parts_10000_uniqs ORDER BY val1 LIMIT 10000000 FORMAT Null + SELECT sum(val2) FROM mt_{parts}_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - DROP TABLE IF EXISTS mt_{parts}_parts + DROP TABLE IF EXISTS mt_{parts}_parts_{uniqs}_uniqs From a08c98d7603e6207ae4a7af2bd78c543602fa76e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 16 Oct 2021 17:03:50 +0300 Subject: [PATCH 750/950] Move some files. --- programs/copier/ClusterCopier.cpp | 4 ++-- programs/library-bridge/Handlers.cpp | 4 ++-- programs/obfuscator/Obfuscator.cpp | 4 ++-- programs/odbc-bridge/MainHandler.cpp | 2 +- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- src/Client/Connection.cpp | 4 ++-- src/Client/IServerConnection.h | 2 +- src/Core/ExternalTable.cpp | 2 +- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- src/Databases/MySQL/FetchTablesColumnsList.cpp | 2 +- src/Databases/MySQL/MaterializeMetadata.cpp | 2 +- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/ClickHouseDictionarySource.cpp | 2 +- src/Dictionaries/DictionaryHelpers.h | 2 +- src/Dictionaries/DirectDictionary.cpp | 2 +- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/IDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 4 ++-- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Dictionaries/PostgreSQLDictionarySource.cpp | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Dictionaries/readInvalidateQuery.cpp | 2 +- src/Formats/TemporaryFileStream.h | 2 +- src/Formats/formatBlock.cpp | 2 +- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- src/Interpreters/SortedBlocksWriter.cpp | 2 +- src/Interpreters/SortedBlocksWriter.h | 2 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Processors/Executors/CompletedPipelineExecutor.cpp | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 2 +- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 +- src/Processors/Executors/PullingPipelineExecutor.cpp | 2 +- src/Processors/Executors/PushingAsyncPipelineExecutor.cpp | 2 +- src/Processors/Executors/PushingPipelineExecutor.cpp | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 2 +- src/Processors/QueryPlan/ArrayJoinStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 +- src/Processors/QueryPlan/CubeStep.cpp | 2 +- src/Processors/QueryPlan/DistinctStep.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 2 +- src/Processors/QueryPlan/ExtremesStep.cpp | 2 +- src/Processors/QueryPlan/FillingStep.cpp | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 2 +- src/Processors/QueryPlan/FinishSortingStep.cpp | 2 +- src/Processors/QueryPlan/ISourceStep.cpp | 2 +- src/Processors/QueryPlan/ITransformingStep.cpp | 2 +- src/Processors/QueryPlan/IntersectOrExceptStep.cpp | 2 +- src/Processors/QueryPlan/JoinStep.cpp | 2 +- src/Processors/QueryPlan/LimitByStep.cpp | 2 +- src/Processors/QueryPlan/LimitStep.cpp | 2 +- src/Processors/QueryPlan/MergeSortingStep.cpp | 2 +- src/Processors/QueryPlan/MergingAggregatedStep.cpp | 2 +- src/Processors/QueryPlan/MergingSortedStep.cpp | 2 +- src/Processors/QueryPlan/OffsetStep.cpp | 2 +- src/Processors/QueryPlan/PartialSortingStep.cpp | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Processors/QueryPlan/ReadFromPreparedSource.cpp | 2 +- src/Processors/QueryPlan/ReadFromPreparedSource.h | 2 +- src/Processors/QueryPlan/ReadNothingStep.cpp | 2 +- src/Processors/QueryPlan/RollupStep.cpp | 2 +- src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp | 2 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 2 +- src/Processors/QueryPlan/UnionStep.cpp | 2 +- src/Processors/QueryPlan/WindowStep.cpp | 2 +- src/Processors/Sources/DelayedSource.h | 2 +- src/Processors/Sources/RemoteSource.h | 2 +- src/Processors/Sources/ShellCommandSource.h | 2 +- src/Processors/Transforms/AggregatingTransform.cpp | 2 +- src/Processors/Transforms/CreatingSetsTransform.h | 4 ++-- .../MergingAggregatedMemoryEfficientTransform.cpp | 2 +- src/Processors/Transforms/buildPushingToViewsChain.h | 2 +- src/Processors/Transforms/getSourceFromASTInsertQuery.cpp | 2 +- src/QueryPipeline/BlockIO.h | 2 +- src/{Processors => QueryPipeline}/Chain.cpp | 2 +- src/{Processors => QueryPipeline}/Chain.h | 2 +- src/{Processors => QueryPipeline}/Pipe.cpp | 2 +- src/{Processors => QueryPipeline}/Pipe.h | 4 ++-- .../PipelineResourcesHolder.cpp | 2 +- src/{Processors => QueryPipeline}/PipelineResourcesHolder.h | 0 src/{Processors => QueryPipeline}/QueryPipeline.cpp | 6 +++--- src/{Processors => QueryPipeline}/QueryPipeline.h | 2 +- src/{Processors => QueryPipeline}/QueryPipelineBuilder.cpp | 2 +- src/{Processors => QueryPipeline}/QueryPipelineBuilder.h | 4 ++-- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- src/QueryPipeline/narrowBlockInputStreams.cpp | 2 +- src/{Processors => QueryPipeline}/printPipeline.cpp | 2 +- src/{Processors => QueryPipeline}/printPipeline.h | 0 .../tests/gtest_blocks_size_merging_streams.cpp | 4 ++-- src/QueryPipeline/tests/gtest_check_sorted_stream.cpp | 4 ++-- src/Server/GRPCServer.cpp | 2 +- src/Storages/Distributed/DistributedSink.h | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 4 ++-- src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 2 +- src/Storages/LiveView/StorageBlocks.h | 2 +- src/Storages/MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/PartitionCommands.cpp | 2 +- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 2 +- src/Storages/ReadFinalForExternalReplicaStorage.h | 2 +- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- src/Storages/StorageDictionary.cpp | 2 +- src/Storages/StorageExecutable.cpp | 2 +- src/Storages/StorageExternalDistributed.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageGenerateRandom.cpp | 2 +- src/Storages/StorageInput.cpp | 2 +- src/Storages/StorageInput.h | 2 +- src/Storages/StorageJoin.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMaterializedMySQL.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StorageNull.h | 2 +- src/Storages/StorageProxy.h | 2 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/StorageS3.cpp | 4 ++-- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageTableFunction.h | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageValues.cpp | 2 +- src/Storages/StorageView.cpp | 2 +- src/Storages/StorageXDBC.cpp | 2 +- src/Storages/System/IStorageSystemOneBlock.h | 2 +- src/Storages/System/StorageSystemNumbers.cpp | 2 +- src/Storages/System/StorageSystemOne.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/System/StorageSystemZeros.cpp | 2 +- src/Storages/tests/gtest_storage_log.cpp | 4 ++-- 141 files changed, 154 insertions(+), 154 deletions(-) rename src/{Processors => QueryPipeline}/Chain.cpp (99%) rename src/{Processors => QueryPipeline}/Chain.h (97%) rename src/{Processors => QueryPipeline}/Pipe.cpp (99%) rename src/{Processors => QueryPipeline}/Pipe.h (98%) rename src/{Processors => QueryPipeline}/PipelineResourcesHolder.cpp (94%) rename src/{Processors => QueryPipeline}/PipelineResourcesHolder.h (100%) rename src/{Processors => QueryPipeline}/QueryPipeline.cpp (99%) rename src/{Processors => QueryPipeline}/QueryPipeline.h (98%) rename src/{Processors => QueryPipeline}/QueryPipelineBuilder.cpp (99%) rename src/{Processors => QueryPipeline}/QueryPipelineBuilder.h (99%) rename src/{Processors => QueryPipeline}/printPipeline.cpp (99%) rename src/{Processors => QueryPipeline}/printPipeline.h (100%) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index e7aeea8cbad..5b21a7c2aef 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/programs/library-bridge/Handlers.cpp b/programs/library-bridge/Handlers.cpp index abc5118baad..bf9ace679ba 100644 --- a/programs/library-bridge/Handlers.cpp +++ b/programs/library-bridge/Handlers.cpp @@ -11,11 +11,11 @@ #include #include #include -#include +#include #include #include #include -#include +#include #include #include diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index f18281a9fdf..caccc726923 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include @@ -25,7 +25,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/programs/odbc-bridge/MainHandler.cpp b/programs/odbc-bridge/MainHandler.cpp index 6362c7dfa9b..82d1bd61c24 100644 --- a/programs/odbc-bridge/MainHandler.cpp +++ b/programs/odbc-bridge/MainHandler.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 9bc14627ac3..e5c6c09ba62 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index a226407e276..21d605e7b45 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -45,7 +45,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 5841c6136af..3c920e8cabe 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -25,8 +25,8 @@ #include "Core/Block.h" #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 7424afc969d..cb76c7ff639 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -8,7 +8,7 @@ #include -#include +#include #include #include diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 4dd8b0cf016..b4adbcc0662 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 83fb101dba8..71b0c2ec6ea 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -13,7 +13,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index ab144761e11..851c9bc74bd 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 8de42760dc2..0facdfc20be 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index d2c1195c0c5..86481d9fd84 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 7683f9d4244..f7e9ce6624c 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include namespace ProfileEvents { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index a19eca1fee7..a5a04d277da 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 6266bd2cf4f..65c40898983 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -16,7 +16,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 03d3b579ec3..f4f33439638 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 532c68d0453..a7cf69bb14d 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include diff --git a/src/Dictionaries/IDictionarySource.h b/src/Dictionaries/IDictionarySource.h index 661f5b8eeb8..5071b69d2bf 100644 --- a/src/Dictionaries/IDictionarySource.h +++ b/src/Dictionaries/IDictionarySource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 5fabe9cf287..f6de6ca0cc1 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -11,8 +11,8 @@ #include "registerDictionaries.h" #include #include -#include -#include +#include +#include #include diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 8d0f0f1abc4..0a9ba1f5ea8 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index a3324b7d769..c9fb8b86b77 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -11,7 +11,7 @@ #include #include "readInvalidateQuery.h" #include -#include +#include #include #endif diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index b0be90c26a5..ba993ec5783 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -16,7 +16,7 @@ #include "readInvalidateQuery.h" #include "registerDictionaries.h" #include -#include +#include #include diff --git a/src/Dictionaries/readInvalidateQuery.cpp b/src/Dictionaries/readInvalidateQuery.cpp index c2c42eece58..370e1457832 100644 --- a/src/Dictionaries/readInvalidateQuery.cpp +++ b/src/Dictionaries/readInvalidateQuery.cpp @@ -1,5 +1,5 @@ #include "readInvalidateQuery.h" -#include +#include #include #include #include diff --git a/src/Formats/TemporaryFileStream.h b/src/Formats/TemporaryFileStream.h index 5a1e0bc870a..4a2aa2d55e0 100644 --- a/src/Formats/TemporaryFileStream.h +++ b/src/Formats/TemporaryFileStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/src/Formats/formatBlock.cpp b/src/Formats/formatBlock.cpp index 3284663fc42..d2b401207aa 100644 --- a/src/Formats/formatBlock.cpp +++ b/src/Formats/formatBlock.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 7e0fa2ba003..cc04c5f013f 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 95b279fd59b..0db07267231 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 487fa2538c2..3afb1e1fbb0 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 59fd1009381..5dbde2344d3 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -33,7 +33,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 7789c74d596..7fb9f1a3ceb 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3d0813579ce..5f08ae6b845 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index ebec58dcca7..1945824636f 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/SortedBlocksWriter.h b/src/Interpreters/SortedBlocksWriter.h index ac58ef2ab7b..0262a274c68 100644 --- a/src/Interpreters/SortedBlocksWriter.h +++ b/src/Interpreters/SortedBlocksWriter.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 45e1c580f01..c3b8cc5c677 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index be388386e9d..a4e3dea89fa 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 0616a392027..ec07cee8738 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 80ed8225c79..fdddfdef2a4 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index ad7da63b0b1..a9c73b9f8fb 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index a1a0755f952..0b6d5334716 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 2e2b5e9ca1e..c1e851e3425 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 023f9016cc4..53036360b8d 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index 35b974baa83..3ca5b9109e6 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 83a4c291bf2..45c3719ebca 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 3d61d3ef36b..23c5115ec68 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index d53d1fa9310..5aeb33fdc7b 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index d1b9c2cad63..33d2ad6e1cf 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ExtremesStep.cpp b/src/Processors/QueryPlan/ExtremesStep.cpp index 117ccd414ca..4524b9883d6 100644 --- a/src/Processors/QueryPlan/ExtremesStep.cpp +++ b/src/Processors/QueryPlan/ExtremesStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 204559ecc3b..223892aa528 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 483055810cf..df75c37dc97 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index c219c09f3bd..6347b69901c 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 61c0a9254cd..0644d9b44eb 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index 1c7f836378f..629fb89be1e 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index a4d81e69fe0..c031303cc7f 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 9c5f8ae2e5f..494a2a6aa0e 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/LimitByStep.cpp b/src/Processors/QueryPlan/LimitByStep.cpp index 12ad933a159..39086e995fc 100644 --- a/src/Processors/QueryPlan/LimitByStep.cpp +++ b/src/Processors/QueryPlan/LimitByStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 3db59e0684a..8c5e3e3c87c 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp index 820bbc31b74..534f05a4d6e 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ b/src/Processors/QueryPlan/MergeSortingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index d02be59ae84..8dfb9f9c923 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index 87d1af4d2bd..ed1f24db55b 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index b48327eb36c..e0c70ba2f28 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp index cf7cb157e4c..420e7b583ca 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ b/src/Processors/QueryPlan/PartialSortingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 6fb6a24f65b..f319e562bfb 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8d3005e725f..cc400aacf2a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index c8213d58db6..fc8136177cf 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.h b/src/Processors/QueryPlan/ReadFromPreparedSource.h index 407b968b728..bb6e814ad9f 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.h +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/ReadNothingStep.cpp b/src/Processors/QueryPlan/ReadNothingStep.cpp index 7019b88f0b2..253f3a5b980 100644 --- a/src/Processors/QueryPlan/ReadNothingStep.cpp +++ b/src/Processors/QueryPlan/ReadNothingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 114fe661c70..acaeb2bc9a7 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp index 47f8187c3aa..4ca3d0ebf54 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index db14950b229..be2cd2348a4 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index 85705595479..5d40a9e241e 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index ca09f4a9474..cd4bb5f6730 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/Sources/DelayedSource.h b/src/Processors/Sources/DelayedSource.h index 7c2b104b61e..3d17c13ad4c 100644 --- a/src/Processors/Sources/DelayedSource.h +++ b/src/Processors/Sources/DelayedSource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Sources/RemoteSource.h b/src/Processors/Sources/RemoteSource.h index 2109cb5eba4..23c3ddec401 100644 --- a/src/Processors/Sources/RemoteSource.h +++ b/src/Processors/Sources/RemoteSource.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Processors/Sources/ShellCommandSource.h b/src/Processors/Sources/ShellCommandSource.h index 18dbd2e03aa..4974c33f290 100644 --- a/src/Processors/Sources/ShellCommandSource.h +++ b/src/Processors/Sources/ShellCommandSource.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index bf3cafd6ff5..8357a997960 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 8e7a09b320d..839ab0cac88 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include #include diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index df2ea4b03f0..7f0893a6c0d 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 6956dedbc41..260fdfb3a19 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 57cfa6c01b8..86998614189 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include "IO/CompressionMethod.h" #include "Parsers/ASTLiteral.h" diff --git a/src/QueryPipeline/BlockIO.h b/src/QueryPipeline/BlockIO.h index d699d525f2f..0f05beca4a8 100644 --- a/src/QueryPipeline/BlockIO.h +++ b/src/QueryPipeline/BlockIO.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Processors/Chain.cpp b/src/QueryPipeline/Chain.cpp similarity index 99% rename from src/Processors/Chain.cpp rename to src/QueryPipeline/Chain.cpp index 5e3b2e6a678..ca7b0b80692 100644 --- a/src/Processors/Chain.cpp +++ b/src/QueryPipeline/Chain.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/src/Processors/Chain.h b/src/QueryPipeline/Chain.h similarity index 97% rename from src/Processors/Chain.h rename to src/QueryPipeline/Chain.h index da5167f9c7a..c5fdc34cecf 100644 --- a/src/Processors/Chain.h +++ b/src/QueryPipeline/Chain.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Processors/Pipe.cpp b/src/QueryPipeline/Pipe.cpp similarity index 99% rename from src/Processors/Pipe.cpp rename to src/QueryPipeline/Pipe.cpp index acf46b95346..6cef7cc28bd 100644 --- a/src/Processors/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Processors/Pipe.h b/src/QueryPipeline/Pipe.h similarity index 98% rename from src/Processors/Pipe.h rename to src/QueryPipeline/Pipe.h index 3341734430c..0af02a5e662 100644 --- a/src/Processors/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -1,8 +1,8 @@ #pragma once #include -#include -#include +#include +#include #include #include diff --git a/src/Processors/PipelineResourcesHolder.cpp b/src/QueryPipeline/PipelineResourcesHolder.cpp similarity index 94% rename from src/Processors/PipelineResourcesHolder.cpp rename to src/QueryPipeline/PipelineResourcesHolder.cpp index 9cb2ea301ad..a4b85ed662b 100644 --- a/src/Processors/PipelineResourcesHolder.cpp +++ b/src/QueryPipeline/PipelineResourcesHolder.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB diff --git a/src/Processors/PipelineResourcesHolder.h b/src/QueryPipeline/PipelineResourcesHolder.h similarity index 100% rename from src/Processors/PipelineResourcesHolder.h rename to src/QueryPipeline/PipelineResourcesHolder.h diff --git a/src/Processors/QueryPipeline.cpp b/src/QueryPipeline/QueryPipeline.cpp similarity index 99% rename from src/Processors/QueryPipeline.cpp rename to src/QueryPipeline/QueryPipeline.cpp index 98ac81f7217..ce1c9473f60 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/QueryPipeline/QueryPipeline.cpp @@ -1,10 +1,10 @@ #include -#include +#include #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Processors/QueryPipeline.h b/src/QueryPipeline/QueryPipeline.h similarity index 98% rename from src/Processors/QueryPipeline.h rename to src/QueryPipeline/QueryPipeline.h index 42fdb429a14..beb46361f95 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/QueryPipeline/QueryPipeline.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp similarity index 99% rename from src/Processors/QueryPipelineBuilder.cpp rename to src/QueryPipeline/QueryPipelineBuilder.cpp index 08a568b48f1..40c64046560 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Processors/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h similarity index 99% rename from src/Processors/QueryPipelineBuilder.h rename to src/QueryPipeline/QueryPipelineBuilder.h index 7e0ddbc0285..d2bbea03ce5 100644 --- a/src/Processors/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index a77a791278a..b01ed7ba9a2 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -7,7 +7,7 @@ #include #include #include "Core/Protocol.h" -#include +#include #include #include #include diff --git a/src/QueryPipeline/narrowBlockInputStreams.cpp b/src/QueryPipeline/narrowBlockInputStreams.cpp index 17e44e38293..19bebe4a0bf 100644 --- a/src/QueryPipeline/narrowBlockInputStreams.cpp +++ b/src/QueryPipeline/narrowBlockInputStreams.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include "narrowBlockInputStreams.h" diff --git a/src/Processors/printPipeline.cpp b/src/QueryPipeline/printPipeline.cpp similarity index 99% rename from src/Processors/printPipeline.cpp rename to src/QueryPipeline/printPipeline.cpp index cbf8cb3a77d..40c88502ed0 100644 --- a/src/Processors/printPipeline.cpp +++ b/src/QueryPipeline/printPipeline.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Processors/printPipeline.h b/src/QueryPipeline/printPipeline.h similarity index 100% rename from src/Processors/printPipeline.h rename to src/QueryPipeline/printPipeline.h diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index 7625fe8fa09..d39c49bb61c 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -4,10 +4,10 @@ #include #include #include -#include +#include #include #include -#include +#include using namespace DB; diff --git a/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp index 04ccc64fd7c..751f7ef8635 100644 --- a/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp +++ b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp @@ -5,8 +5,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 25085f38889..ba2644e0fba 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -32,7 +32,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 3c0b6333fc1..668cec22e8b 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 12558054d81..6e81f5577ab 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -24,8 +24,8 @@ #include #include #include -#include -#include +#include +#include #include diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 8effb5ed9bf..7c158794caf 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index ec6e58f3fbb..74e17442fe8 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/LiveView/StorageBlocks.h b/src/Storages/LiveView/StorageBlocks.h index 6cf7ce59fa2..f010ef50ecd 100644 --- a/src/Storages/LiveView/StorageBlocks.h +++ b/src/Storages/LiveView/StorageBlocks.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 23ea280e88c..729b545e9a0 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 917ea156ab7..fa36588513b 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 4e9e2c450b1..f771b2239ef 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 42294b8152c..19f1d7780b0 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/ReadFinalForExternalReplicaStorage.h b/src/Storages/ReadFinalForExternalReplicaStorage.h index f09a115919d..1be0aa0f4a6 100644 --- a/src/Storages/ReadFinalForExternalReplicaStorage.h +++ b/src/Storages/ReadFinalForExternalReplicaStorage.h @@ -7,7 +7,7 @@ #if USE_MYSQL || USE_LIBPQXX #include -#include +#include namespace DB diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 3a37e57e008..18bf0e2c19b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -17,7 +17,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index e305d4c6183..349d894b2f1 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 21784952c23..76be3353808 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index cedb7d26b48..0af8fe6f7df 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 4ae55272db6..7c6543c5519 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -36,7 +36,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index cfec328bbfc..a4dfbfc3f96 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageInput.cpp b/src/Storages/StorageInput.cpp index 1138794adb0..2ed7a77b59d 100644 --- a/src/Storages/StorageInput.cpp +++ b/src/Storages/StorageInput.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Storages/StorageInput.h b/src/Storages/StorageInput.h index bfb634c6bba..b28bc143bb0 100644 --- a/src/Storages/StorageInput.h +++ b/src/Storages/StorageInput.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index c2c1cff5f53..8ac341661bf 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -17,7 +17,7 @@ #include #include -#include +#include #include #include /// toLower diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ecf3da83ea6..41d178fc04d 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -23,7 +23,7 @@ #include "StorageLogSettings.h" #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index 52f53b9ceee..37a9838593b 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -18,7 +18,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 299e39a3836..37cb238ba0f 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ab42da1dfa0..acc40787fed 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -27,7 +27,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 15430f60285..146fd8b616a 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index a1724af38cb..50d2124958c 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h index 96e72c77f00..de667c1d75c 100644 --- a/src/Storages/StorageNull.h +++ b/src/Storages/StorageNull.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h index c81ef6febdc..62e0ccc76aa 100644 --- a/src/Storages/StorageProxy.h +++ b/src/Storages/StorageProxy.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 65e07fa6144..bc03dbb45ae 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -30,7 +30,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index c94c519c3b9..8cbec46d0e0 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -35,7 +35,7 @@ #include #include -#include +#include #include #include @@ -52,7 +52,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index b78a09ebcc4..9d05235552c 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -24,7 +24,7 @@ #include #include #include -#include +#include #include "Processors/Sources/SourceWithProgress.h" #include #include diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index cc5eb835438..50113c391cc 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -31,7 +31,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageTableFunction.h b/src/Storages/StorageTableFunction.h index 557f378ab77..0b7ab30fa24 100644 --- a/src/Storages/StorageTableFunction.h +++ b/src/Storages/StorageTableFunction.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index d1778342ec5..74d922d5594 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -36,7 +36,7 @@ #include #include -#include +#include #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9397986fadd..174ee58ee42 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -19,7 +19,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index ace5ca3667c..650782afbba 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index df774554365..9ffea587b97 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -14,7 +14,7 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a0924896437..a90e21a2edb 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index c074659af2b..6897b4a3a79 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 3a88cc96639..136c2489be2 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Storages/System/StorageSystemOne.cpp b/src/Storages/System/StorageSystemOne.cpp index 7c28f897121..7558ae0ae92 100644 --- a/src/Storages/System/StorageSystemOne.cpp +++ b/src/Storages/System/StorageSystemOne.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 254e6f77e0c..f32a609077f 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index d1456d72685..624fc54998c 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 5b891c43aae..f86295cd06b 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -18,8 +18,8 @@ #include #include #include -#include -#include +#include +#include #if !defined(__clang__) # pragma GCC diagnostic push From fa92fd28bf23dd9a0522020d44ee1cb67e08679b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sat, 16 Oct 2021 17:34:31 +0300 Subject: [PATCH 751/950] Update LocalServer.cpp --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 10c4bf592ca..766123339af 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -129,7 +129,7 @@ bool LocalServer::executeMultiQuery(const String & all_queries_text) { if (current_exception) current_exception->rethrow(); - continue; + return true; } case MultiQueryProcessingStage::EXECUTE_QUERY: { From 3102d24872e1579dd28d2133171cb6e2d4aceccc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 15 Oct 2021 10:38:11 +0300 Subject: [PATCH 752/950] Fix --hung-check in clickhouse-test JSONEachRow cannot be parsed with a simple json.loads(), instead it should be passed to json.loads() line by line. Fixes: #30065 --- tests/clickhouse-test | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b2a9358371a..7b934844dc4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -109,7 +109,10 @@ def clickhouse_execute_json(base_args, query, timeout=30, settings=None): data = clickhouse_execute_http(base_args, query, timeout, settings, 'JSONEachRow') if not data: return None - return json.loads(data) + rows = [] + for row in data.strip().split(b'\n'): + rows.append(json.loads(row)) + return rows class Terminated(KeyboardInterrupt): From c6e61e9497f708589eaf0602a9727943c015190b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Oct 2021 17:58:19 +0300 Subject: [PATCH 753/950] clickhouse-test: use splitlines() over split('\n') --- tests/clickhouse-test | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7b934844dc4..6512c40c44b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -110,7 +110,7 @@ def clickhouse_execute_json(base_args, query, timeout=30, settings=None): if not data: return None rows = [] - for row in data.strip().split(b'\n'): + for row in data.strip().splitlines(): rows.append(json.loads(row)) return rows @@ -478,19 +478,19 @@ class TestCase: if os.path.isfile(self.stdout_file): description += ", result:\n\n" - description += '\n'.join(open(self.stdout_file).read().split('\n')[:100]) + description += '\n'.join(open(self.stdout_file).read().splitlines()[:100]) description += '\n' description += "\nstdout:\n{}\n".format(stdout) return TestResult(self.name, TestStatus.FAIL, reason, total_time, description) if stderr: - description += "\n{}\n".format('\n'.join(stderr.split('\n')[:100])) + description += "\n{}\n".format('\n'.join(stderr.splitlines()[:100])) description += "\nstdout:\n{}\n".format(stdout) return TestResult(self.name, TestStatus.FAIL, FailureReason.STDERR, total_time, description) if 'Exception' in stdout: - description += "\n{}\n".format('\n'.join(stdout.split('\n')[:100])) + description += "\n{}\n".format('\n'.join(stdout.splitlines()[:100])) return TestResult(self.name, TestStatus.FAIL, FailureReason.EXCEPTION, total_time, description) if '@@SKIP@@' in stdout: From 612a21cc9775590900c40ebe987da6d9571b2cf2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Oct 2021 17:59:17 +0300 Subject: [PATCH 754/950] clickhouse-test: do not propagate CLICKHOUSE_PORT_HTTP to clickhouse-client --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6512c40c44b..1e8c2bfb8ad 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1395,7 +1395,6 @@ if __name__ == '__main__': http_port = os.getenv("CLICKHOUSE_PORT_HTTP") if http_port is not None: args.http_port = int(http_port) - args.client += f" --port={http_port}" else: args.http_port = 8123 From 6752be4c4ea3f8d7ee3056f6927d7f966c103de4 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sat, 16 Oct 2021 23:11:45 +0800 Subject: [PATCH 755/950] fix bug and add test --- src/Functions/ReplaceRegexpImpl.h | 16 ++++++++++++++-- .../02100_replaceRegexpAll_bug.reference | 3 +++ .../0_stateless/02100_replaceRegexpAll_bug.sql | 3 +++ 3 files changed, 20 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference create mode 100644 tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 1caced9cbde..7662b747feb 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -96,6 +96,9 @@ struct ReplaceRegexpImpl re2_st::StringPiece matches[max_captures]; size_t start_pos = 0; + bool is_first_match = true; + bool is_start_pos_added_one = false; + while (start_pos < static_cast(input.length())) { /// If no more replacements possible for current string @@ -103,6 +106,9 @@ struct ReplaceRegexpImpl if (searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) { + if (is_start_pos_added_one) + start_pos -= 1; + const auto & match = matches[0]; size_t bytes_to_copy = (match.data() - input.data()) - start_pos; @@ -110,7 +116,12 @@ struct ReplaceRegexpImpl res_data.resize(res_data.size() + bytes_to_copy); memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, bytes_to_copy); res_offset += bytes_to_copy; - start_pos += bytes_to_copy + (match.length() > 0 ? match.length() : 1); + start_pos += bytes_to_copy + match.length(); + if (is_first_match && match.length() == 0 && !replace_one && input.length() > 1) + { + start_pos += 1; + is_start_pos_added_one = true; + } /// Do substitution instructions for (const auto & it : instructions) @@ -129,8 +140,9 @@ struct ReplaceRegexpImpl } } - if (replace_one) /// Stop after match of zero length, to avoid infinite loop. + if (replace_one || !is_first_match) /// Stop after match of zero length, to avoid infinite loop. can_finish_current_string = true; + is_first_match = false; } else can_finish_current_string = true; diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference new file mode 100644 index 00000000000..2bb40778ca6 --- /dev/null +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference @@ -0,0 +1,3 @@ +aaaabb +b aaaa +aaaa diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql new file mode 100644 index 00000000000..d0caeacfa0e --- /dev/null +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql @@ -0,0 +1,3 @@ +SELECT trim(leading 'b ' FROM 'b aaaabb ') x; +SELECT trim(trailing 'b ' FROM 'b aaaabb ') x; +SELECT trim(both 'b ' FROM 'b aaaabb ') x; From 1541593b2f6cb6e991af22b5a1885f6848f908b1 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sat, 16 Oct 2021 23:28:29 +0800 Subject: [PATCH 756/950] add notes --- src/Functions/ReplaceRegexpImpl.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 7662b747feb..a297be42aaf 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -117,6 +117,8 @@ struct ReplaceRegexpImpl memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, bytes_to_copy); res_offset += bytes_to_copy; start_pos += bytes_to_copy + match.length(); + + /// To avoid infinite loop. if (is_first_match && match.length() == 0 && !replace_one && input.length() > 1) { start_pos += 1; @@ -140,7 +142,7 @@ struct ReplaceRegexpImpl } } - if (replace_one || !is_first_match) /// Stop after match of zero length, to avoid infinite loop. + if (replace_one || !is_first_match) can_finish_current_string = true; is_first_match = false; } From 434de66a2b6f34eb8f8fc4704eded45bb0763fd0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 16 Oct 2021 12:59:01 -0300 Subject: [PATCH 757/950] Doc. ArgMax/Min are not supported anymore by SimpleAggregateFunction --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 7b81c577762..8f47bd0902b 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -21,8 +21,6 @@ - [`sumMap`](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) - [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap) - [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap) -- [`argMin`](../../sql-reference/aggregate-functions/reference/argmin.md) -- [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md) !!! note "Примечание" Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому комбинаторы [-Merge](../../sql-reference/aggregate-functions/combinators.md#aggregate_functions_combinators-merge) и [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) не требуются. From bd2c016cf32eb0ca9649c289baee07032294e480 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sun, 17 Oct 2021 00:33:57 +0800 Subject: [PATCH 758/950] fix bug --- src/Functions/ReplaceRegexpImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index a297be42aaf..b2c5470cd96 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -142,7 +142,7 @@ struct ReplaceRegexpImpl } } - if (replace_one || !is_first_match) + if (replace_one || (!is_first_match && match.length() == 0)) can_finish_current_string = true; is_first_match = false; } From 329d81a0efb7fab70fe91a8da5cd0ee36c47ad82 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 16 Oct 2021 20:03:45 +0300 Subject: [PATCH 759/950] Various fixes to install procedure --- programs/install/Install.cpp | 516 +++++++++++++++++++++-------------- tests/CMakeLists.txt | 22 +- 2 files changed, 323 insertions(+), 215 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index e001d0ceb53..6b2d3a58a22 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -66,6 +66,7 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int SYSTEM_ERROR; extern const int NOT_ENOUGH_SPACE; + extern const int NOT_IMPLEMENTED; extern const int CANNOT_KILL; } @@ -75,8 +76,17 @@ namespace ErrorCodes #define HILITE "\033[1m" #define END_HILITE "\033[0m" -static constexpr auto CLICKHOUSE_BRIDGE_USER = "clickhouse-bridge"; -static constexpr auto CLICKHOUSE_BRIDGE_GROUP = "clickhouse-bridge"; +#if defined(OS_DARWIN) +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = ""; +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_GROUP = ""; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_USER = ""; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_GROUP = ""; +#else +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = "clickhouse"; +static constexpr auto DEFAULT_CLICKHOUSE_SERVER_GROUP = "clickhouse"; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_USER = "clickhouse-bridge"; +static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_GROUP = "clickhouse-bridge"; +#endif using namespace DB; namespace po = boost::program_options; @@ -127,36 +137,77 @@ static bool filesEqual(std::string path1, std::string path2) && 0 == memcmp(in1.buffer().begin(), in2.buffer().begin(), in1.buffer().size()); } +static void changeOwnership(const String & file_name, const String & user_name, const String & group_name = {}, bool recursive = true) +{ + if (!user_name.empty() || !group_name.empty()) + { + std::string command = fmt::format("chown {} {}:{} '{}'", (recursive ? "-R" : ""), user_name, group_name, file_name); + fmt::print(" {}\n", command); + executeScript(command); + } +} + +static void createGroup(const String & group_name) +{ + if (!group_name.empty()) + { +#if defined(OS_DARWIN) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a group in macOS"); +#else + std::string command = fmt::format("groupadd -r {}", group_name); + fmt::print(" {}\n", command); + executeScript(command); +#endif + } +} + +static void createUser(const String & user_name, [[maybe_unused]] const String & group_name) +{ + if (!user_name.empty()) + { +#if defined(OS_DARWIN) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a user in macOS"); +#else + std::string command = group_name.empty() + ? fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent --user-group {}", user_name) + : fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent -g {} {}", group_name, user_name); + fmt::print(" {}\n", command); + executeScript(command); +#endif + } +} + int mainEntryClickHouseInstall(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("prefix", po::value()->default_value(""), "prefix for all paths") - ("binary-path", po::value()->default_value("/usr/bin"), "where to install binaries") - ("config-path", po::value()->default_value("/etc/clickhouse-server"), "where to install configs") - ("log-path", po::value()->default_value("/var/log/clickhouse-server"), "where to create log directory") - ("data-path", po::value()->default_value("/var/lib/clickhouse"), "directory for data") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("user", po::value()->default_value("clickhouse"), "clickhouse user to create") - ("group", po::value()->default_value("clickhouse"), "clickhouse group to create") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " install [options]\n"; - std::cout << desc << '\n'; - } - try { + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("binary-path", po::value()->default_value("usr/bin"), "where to install binaries") + ("config-path", po::value()->default_value("etc/clickhouse-server"), "where to install configs") + ("log-path", po::value()->default_value("var/log/clickhouse-server"), "where to create log directory") + ("data-path", po::value()->default_value("var/lib/clickhouse"), "directory for data") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user to create") + ("group", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_GROUP), "clickhouse group to create") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " install [options]\n"; + std::cout << desc << '\n'; + return 1; + } + /// We need to copy binary to the binary directory. /// The binary is currently run. We need to obtain its path from procfs (on Linux). @@ -171,6 +222,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (res != 0) Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot obtain path to the binary"); + if (path.back() == '\0') + path.pop_back(); + fs::path binary_self_path(path); #else fs::path binary_self_path = "/proc/self/exe"; @@ -186,8 +240,8 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// TODO An option to link instead of copy - useful for developers. - fs::path prefix = fs::path(options["prefix"].as()); - fs::path bin_dir = prefix / fs::path(options["binary-path"].as()); + fs::path prefix = options["prefix"].as(); + fs::path bin_dir = prefix / options["binary-path"].as(); fs::path main_bin_path = bin_dir / "clickhouse"; fs::path main_bin_tmp_path = bin_dir / "clickhouse.new"; @@ -225,6 +279,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { + if (!fs::exists(bin_dir)) + { + fmt::print("Creating binary directory {}.\n", bin_dir.string()); + fs::create_directories(bin_dir); + } + size_t available_space = fs::space(bin_dir).available; if (available_space < binary_size) throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space for clickhouse binary in {}, required {}, available {}.", @@ -326,34 +386,18 @@ int mainEntryClickHouseInstall(int argc, char ** argv) std::string user = options["user"].as(); std::string group = options["group"].as(); - auto create_group = [](const String & group_name) - { - std::string command = fmt::format("groupadd -r {}", group_name); - fmt::print(" {}\n", command); - executeScript(command); - }; - if (!group.empty()) { fmt::print("Creating clickhouse group if it does not exist.\n"); - create_group(group); + createGroup(group); } else - fmt::print("Will not create clickhouse group"); - - auto create_user = [](const String & user_name, const String & group_name) - { - std::string command = group_name.empty() - ? fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent --user-group {}", user_name) - : fmt::format("useradd -r --shell /bin/false --home-dir /nonexistent -g {} {}", group_name, user_name); - fmt::print(" {}\n", command); - executeScript(command); - }; + fmt::print("Will not create a dedicated clickhouse group.\n"); if (!user.empty()) { fmt::print("Creating clickhouse user if it does not exist.\n"); - create_user(user, group); + createUser(user, group); if (group.empty()) group = user; @@ -361,6 +405,11 @@ int mainEntryClickHouseInstall(int argc, char ** argv) /// Setting ulimits. try { +#if defined(OS_DARWIN) + + /// TODO Set ulimits on macOS. + +#else fs::path ulimits_dir = "/etc/security/limits.d"; fs::path ulimits_file = ulimits_dir / fmt::format("{}.conf", user); fmt::print("Will set ulimits for {} user in {}.\n", user, ulimits_file.string()); @@ -374,16 +423,15 @@ int mainEntryClickHouseInstall(int argc, char ** argv) out.write(ulimits_content.data(), ulimits_content.size()); out.sync(); out.finalize(); +#endif } catch (...) { std::cerr << "Cannot set ulimits: " << getCurrentExceptionMessage(false) << "\n"; } - - /// TODO Set ulimits on Mac OS X } else - fmt::print("Will not create clickhouse user.\n"); + fmt::print("Will not create a dedicated clickhouse user.\n"); /// Creating configuration files and directories. @@ -400,9 +448,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) fs::path config_d = config_dir / "config.d"; fs::path users_d = config_dir / "users.d"; - std::string log_path = prefix / options["log-path"].as(); - std::string data_path = prefix / options["data-path"].as(); - std::string pid_path = prefix / options["pid-path"].as(); + fs::path log_path = prefix / options["log-path"].as(); + fs::path data_path = prefix / options["data-path"].as(); + fs::path pid_path = prefix / options["pid-path"].as(); bool has_password_for_default_user = false; @@ -427,10 +475,78 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - WriteBufferFromFile out(main_config_file.string()); - out.write(main_config_content.data(), main_config_content.size()); - out.sync(); - out.finalize(); + { + WriteBufferFromFile out(main_config_file.string()); + out.write(main_config_content.data(), main_config_content.size()); + out.sync(); + out.finalize(); + } + + /// Override the default paths. + + /// Data paths. + { + std::string data_file = config_d / "data-paths.xml"; + WriteBufferFromFile out(data_file); + out << "\n" + " " << data_path.string() << "\n" + " " << (data_path / "tmp").string() << "\n" + " " << (data_path / "user_files").string() << "\n" + " " << (data_path / "format_schemas").string() << "\n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("Data path configuration override is saved to file {}.\n", data_file); + } + + /// Logger. + { + std::string logger_file = config_d / "logger.xml"; + WriteBufferFromFile out(logger_file); + out << "\n" + " \n" + " " << (log_path / "clickhouse-server.log").string() << "\n" + " " << (log_path / "clickhouse-server.err.log").string() << "\n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("Log path configuration override is saved to file {}.\n", logger_file); + } + + /// User directories. + { + std::string user_directories_file = config_d / "user-directories.xml"; + WriteBufferFromFile out(user_directories_file); + out << "\n" + " \n" + " \n" + " " << (data_path / "access").string() << "\n" + " \n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("User directory path configuration override is saved to file {}.\n", user_directories_file); + } + + /// OpenSSL. + { + std::string openssl_file = config_d / "openssl.xml"; + WriteBufferFromFile out(openssl_file); + out << "\n" + " \n" + " \n" + " " << (config_dir / "server.crt").string() << "\n" + " " << (config_dir / "server.key").string() << "\n" + " " << (config_dir / "dhparam.pem").string() << "\n" + " \n" + " \n" + "\n"; + out.sync(); + out.finalize(); + fmt::print("OpenSSL path configuration override is saved to file {}.\n", openssl_file); + } } } else @@ -443,13 +559,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (configuration->has("path")) { data_path = configuration->getString("path"); - fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); + fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path.string()); } if (configuration->has("logger.log")) { log_path = fs::path(configuration->getString("logger.log")).remove_filename(); - fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); + fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path.string()); } } @@ -485,82 +601,44 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } - auto change_ownership = [](const String & file_name, const String & user_name, const String & group_name) - { - std::string command = fmt::format("chown --recursive {}:{} '{}'", user_name, group_name, file_name); - fmt::print(" {}\n", command); - executeScript(command); - }; - - /// Chmod and chown configs - change_ownership(config_dir.string(), user, group); - - /// Symlink "preprocessed_configs" is created by the server, so "write" is needed. - fs::permissions(config_dir, fs::perms::owner_all, fs::perm_options::replace); - - /// Subdirectories, so "execute" is needed. - if (fs::exists(config_d)) - fs::permissions(config_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); - if (fs::exists(users_d)) - fs::permissions(users_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); - - /// Readonly. - if (fs::exists(main_config_file)) - fs::permissions(main_config_file, fs::perms::owner_read, fs::perm_options::replace); - if (fs::exists(users_config_file)) - fs::permissions(users_config_file, fs::perms::owner_read, fs::perm_options::replace); - /// Create directories for data and log. if (fs::exists(log_path)) { - fmt::print("Log directory {} already exists.\n", log_path); + fmt::print("Log directory {} already exists.\n", log_path.string()); } else { - fmt::print("Creating log directory {}.\n", log_path); + fmt::print("Creating log directory {}.\n", log_path.string()); fs::create_directories(log_path); } if (fs::exists(data_path)) { - fmt::print("Data directory {} already exists.\n", data_path); + fmt::print("Data directory {} already exists.\n", data_path.string()); } else { - fmt::print("Creating data directory {}.\n", data_path); + fmt::print("Creating data directory {}.\n", data_path.string()); fs::create_directories(data_path); } if (fs::exists(pid_path)) { - fmt::print("Pid directory {} already exists.\n", pid_path); + fmt::print("Pid directory {} already exists.\n", pid_path.string()); } else { - fmt::print("Creating pid directory {}.\n", pid_path); + fmt::print("Creating pid directory {}.\n", pid_path.string()); fs::create_directories(pid_path); } /// Chmod and chown data and log directories - { - std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, log_path); - fmt::print(" {}\n", command); - executeScript(command); - } + changeOwnership(log_path, user, group); + changeOwnership(pid_path, user, group); - { - std::string command = fmt::format("chown --recursive {}:{} '{}'", user, group, pid_path); - fmt::print(" {}\n", command); - executeScript(command); - } - - { - /// Not recursive, because there can be a huge number of files and it will be slow. - std::string command = fmt::format("chown {}:{} '{}'", user, group, data_path); - fmt::print(" {}\n", command); - executeScript(command); - } + /// Not recursive, because there can be a huge number of files and it will be slow. + changeOwnership(data_path, user, group, /* recursive= */ false); /// All users are allowed to read pid file (for clickhouse status command). fs::permissions(pid_path, fs::perms::owner_all | fs::perms::group_read | fs::perms::others_read, fs::perm_options::replace); @@ -576,13 +654,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (fs::exists(odbc_bridge_path) || fs::exists(library_bridge_path)) { - create_group(CLICKHOUSE_BRIDGE_GROUP); - create_user(CLICKHOUSE_BRIDGE_USER, CLICKHOUSE_BRIDGE_GROUP); + createGroup(DEFAULT_CLICKHOUSE_BRIDGE_GROUP); + createUser(DEFAULT_CLICKHOUSE_BRIDGE_USER, DEFAULT_CLICKHOUSE_BRIDGE_GROUP); if (fs::exists(odbc_bridge_path)) - change_ownership(odbc_bridge_path, CLICKHOUSE_BRIDGE_USER, CLICKHOUSE_BRIDGE_GROUP); + changeOwnership(odbc_bridge_path, DEFAULT_CLICKHOUSE_BRIDGE_USER, DEFAULT_CLICKHOUSE_BRIDGE_GROUP); if (fs::exists(library_bridge_path)) - change_ownership(library_bridge_path, CLICKHOUSE_BRIDGE_USER, CLICKHOUSE_BRIDGE_GROUP); + changeOwnership(library_bridge_path, DEFAULT_CLICKHOUSE_BRIDGE_USER, DEFAULT_CLICKHOUSE_BRIDGE_GROUP); } bool stdin_is_a_tty = isatty(STDIN_FILENO); @@ -701,6 +779,25 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } } + /// Chmod and chown configs + changeOwnership(config_dir, user, group); + + /// Symlink "preprocessed_configs" is created by the server, so "write" is needed. + fs::permissions(config_dir, fs::perms::owner_all, fs::perm_options::replace); + + /// Subdirectories, so "execute" is needed. + if (fs::exists(config_d)) + fs::permissions(config_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); + if (fs::exists(users_d)) + fs::permissions(users_d, fs::perms::owner_read | fs::perms::owner_exec, fs::perm_options::replace); + + /// Readonly. + if (fs::exists(main_config_file)) + fs::permissions(main_config_file, fs::perms::owner_read, fs::perm_options::replace); + if (fs::exists(users_config_file)) + fs::permissions(users_config_file, fs::perms::owner_read, fs::perm_options::replace); + + std::string maybe_password; if (has_password_for_default_user) maybe_password = " --password"; @@ -766,11 +863,7 @@ namespace /// All users are allowed to read pid file (for clickhouse status command). fs::permissions(pid_path, fs::perms::owner_all | fs::perms::group_read | fs::perms::others_read, fs::perm_options::replace); - { - std::string command = fmt::format("chown --recursive {} '{}'", user, pid_path.string()); - fmt::print(" {}\n", command); - executeScript(command); - } + changeOwnership(pid_path, user); } std::string command = fmt::format("{} --config-file {} --pid-file {} --daemon", @@ -974,34 +1067,36 @@ namespace int mainEntryClickHouseStart(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("binary-path", po::value()->default_value("/usr/bin"), "directory with binary") - ("config-path", po::value()->default_value("/etc/clickhouse-server"), "directory with configs") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("user", po::value()->default_value("clickhouse"), "clickhouse user") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " start\n"; - return 1; - } - try { + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("binary-path", po::value()->default_value("usr/bin"), "directory with binary") + ("config-path", po::value()->default_value("etc/clickhouse-server"), "directory with configs") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " start\n"; + return 1; + } + std::string user = options["user"].as(); - fs::path executable = fs::path(options["binary-path"].as()) / "clickhouse-server"; - fs::path config = fs::path(options["config-path"].as()) / "config.xml"; - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + fs::path prefix = options["prefix"].as(); + fs::path executable = prefix / options["binary-path"].as() / "clickhouse-server"; + fs::path config = prefix / options["config-path"].as() / "config.xml"; + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; return start(user, executable, config, pid_file); } @@ -1015,28 +1110,30 @@ int mainEntryClickHouseStart(int argc, char ** argv) int mainEntryClickHouseStop(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("force", po::bool_switch(), "Stop with KILL signal instead of TERM") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " stop\n"; - return 1; - } - try { - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("force", po::bool_switch(), "Stop with KILL signal instead of TERM") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " stop\n"; + return 1; + } + + fs::path prefix = options["prefix"].as(); + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; return stop(pid_file, options["force"].as()); } @@ -1050,72 +1147,79 @@ int mainEntryClickHouseStop(int argc, char ** argv) int mainEntryClickHouseStatus(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " status\n"; - return 1; - } - try { - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " status\n"; + return 1; + } + + fs::path prefix = options["prefix"].as(); + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; + isRunning(pid_file); - return 0; } catch (...) { std::cerr << getCurrentExceptionMessage(false) << '\n'; return getCurrentExceptionCode(); } + + return 0; } int mainEntryClickHouseRestart(int argc, char ** argv) { - po::options_description desc; - desc.add_options() - ("help,h", "produce help message") - ("binary-path", po::value()->default_value("/usr/bin"), "directory with binary") - ("config-path", po::value()->default_value("/etc/clickhouse-server"), "directory with configs") - ("pid-path", po::value()->default_value("/var/run/clickhouse-server"), "directory for pid file") - ("user", po::value()->default_value("clickhouse"), "clickhouse user") - ("force", po::value()->default_value(false), "Stop with KILL signal instead of TERM") - ; - - po::variables_map options; - po::store(po::parse_command_line(argc, argv, desc), options); - - if (options.count("help")) - { - std::cout << "Usage: " - << (getuid() == 0 ? "" : "sudo ") - << argv[0] - << " restart\n"; - return 1; - } - try { + po::options_description desc; + desc.add_options() + ("help,h", "produce help message") + ("prefix", po::value()->default_value("/"), "prefix for all paths") + ("binary-path", po::value()->default_value("usr/bin"), "directory with binary") + ("config-path", po::value()->default_value("etc/clickhouse-server"), "directory with configs") + ("pid-path", po::value()->default_value("var/run/clickhouse-server"), "directory for pid file") + ("user", po::value()->default_value(DEFAULT_CLICKHOUSE_SERVER_USER), "clickhouse user") + ("force", po::value()->default_value(false), "Stop with KILL signal instead of TERM") + ; + + po::variables_map options; + po::store(po::parse_command_line(argc, argv, desc), options); + + if (options.count("help")) + { + std::cout << "Usage: " + << (getuid() == 0 ? "" : "sudo ") + << argv[0] + << " restart\n"; + return 1; + } + std::string user = options["user"].as(); - fs::path executable = fs::path(options["binary-path"].as()) / "clickhouse-server"; - fs::path config = fs::path(options["config-path"].as()) / "config.xml"; - fs::path pid_file = fs::path(options["pid-path"].as()) / "clickhouse-server.pid"; + fs::path prefix = options["prefix"].as(); + fs::path executable = prefix / options["binary-path"].as() / "clickhouse-server"; + fs::path config = prefix / options["config-path"].as() / "config.xml"; + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; if (int res = stop(pid_file, options["force"].as())) return res; + return start(user, executable, config, pid_file); } catch (...) diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 132909438da..c9858910837 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -7,15 +7,19 @@ else () include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake) endif () -install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) -install ( - DIRECTORY queries performance config - DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test - USE_SOURCE_PERMISSIONS - COMPONENT clickhouse - PATTERN "CMakeLists.txt" EXCLUDE - PATTERN ".gitignore" EXCLUDE -) +option (ENABLE_CLICKHOUSE_TEST "Install clickhouse-test script and relevant tests scenarios" ON) + +if (ENABLE_CLICKHOUSE_TEST) + install (PROGRAMS clickhouse-test DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) + install ( + DIRECTORY queries performance config + DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse-test + USE_SOURCE_PERMISSIONS + COMPONENT clickhouse + PATTERN "CMakeLists.txt" EXCLUDE + PATTERN ".gitignore" EXCLUDE + ) +endif () if (ENABLE_TEST_INTEGRATION) add_subdirectory (integration) From 662b5d40e97c36259edff267595129d0fc28f396 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Sat, 16 Oct 2021 20:35:41 +0200 Subject: [PATCH 760/950] Adjust the tests to do less work --- ...k_many_parallel_quorum_inserts_long.reference | 16 ++++++---------- ...09_check_many_parallel_quorum_inserts_long.sh | 12 ++++++------ .../02033_join_engine_deadlock_long.sh | 4 ++-- 3 files changed, 14 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.reference b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.reference index 52dea650ebc..e9b7db9d530 100644 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.reference +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.reference @@ -1,10 +1,6 @@ -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 +30 0 54 810 +30 0 54 810 +30 0 54 810 +30 0 54 810 +30 0 54 810 +30 0 54 810 diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index a6f151d0f6f..030ae017e71 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-replicated-database, no-parallel +# Tags: long, no-replicated-database # Tag no-replicated-database: Fails due to additional replicas or shards set -e @@ -8,7 +8,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -NUM_REPLICAS=10 +NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " @@ -20,17 +20,17 @@ done function thread { i=0 retries=300 while [[ $i -lt $retries ]]; do # server can be dead - $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break + $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break ((++i)) sleep 0.1 done } for i in $(seq 1 $NUM_REPLICAS); do - for j in {0..9}; do + for j in {0..4}; do a=$((($i - 1) * 10 + $j)) - # Note: making 100 connections simultaneously is a mini-DoS when server is build with sanitizers and CI environment is overloaded. + # Note: making 30 connections simultaneously is a mini-DoS when server is build with sanitizers and CI environment is overloaded. # That's why we repeat "socket timeout" errors. thread $i $a 2>&1 | grep -v -P 'SOCKET_TIMEOUT|NETWORK_ERROR|^$' & done @@ -46,5 +46,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh index 1e670b72fe7..13cf013b53b 100755 --- a/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh +++ b/tests/queries/0_stateless/02033_join_engine_deadlock_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, deadlock, no-parallel +# Tags: long, deadlock CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -29,7 +29,7 @@ populate_table_bg () { $CLICKHOUSE_CLIENT --query " INSERT INTO join_block_test SELECT toString(number) as id, number * number as num - FROM system.numbers LIMIT 3000000 + FROM system.numbers LIMIT 500000 " >/dev/null ) & } From d34d752688ec706694a05c5fd0c568c651b57c14 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 18:37:46 +0000 Subject: [PATCH 761/950] Fix tests --- programs/local/LocalServer.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index cdd5ae13f99..0c5f64ea913 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -579,6 +579,11 @@ void LocalServer::processConfig() { String path = global_context->getPath(); + /// When tables are loaded from .sql we initialize background executors + /// regardless there are MergeTree tables or not, because no better place was found. + /// In other cases it will be initialized only when there are mergeTree tables. + global_context->initializeBackgroundExecutors(); + /// Lock path directory before read status.emplace(fs::path(path) / "status", StatusFile::write_full_info); From 8b0359584afa84a310f43c37aeb01caf1ec7450a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 19:08:37 +0000 Subject: [PATCH 762/950] Update test --- tests/queries/0_stateless/01196_max_parser_depth.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01196_max_parser_depth.reference b/tests/queries/0_stateless/01196_max_parser_depth.reference index a72c1b18aa2..072fc270acd 100644 --- a/tests/queries/0_stateless/01196_max_parser_depth.reference +++ b/tests/queries/0_stateless/01196_max_parser_depth.reference @@ -1,3 +1,4 @@ Code: 306 Code: 306 Code: 306 +Code: 306 From 443efe8f5c207689f0575d3c79b03d9589b2d191 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 16 Oct 2021 16:21:18 -0300 Subject: [PATCH 763/950] Update quotas.md --- docs/ru/operations/system-tables/quotas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/system-tables/quotas.md b/docs/ru/operations/system-tables/quotas.md index 3715bc89596..6c8b5a3eebf 100644 --- a/docs/ru/operations/system-tables/quotas.md +++ b/docs/ru/operations/system-tables/quotas.md @@ -11,7 +11,7 @@ - `[]` — Все пользователи используют одну и ту же квоту. - `['user_name']` — Соединения с одинаковым именем пользователя используют одну и ту же квоту. - `['ip_address']` — Соединения с одинаковым IP-адресом используют одну и ту же квоту. - - `['client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Ключ может быть явно задан клиентом. При использовании [clickhouse-client](../../interfaces/cli.md), передайте ключевое значение в параметре `--quota-key`, или используйте параметр `quota_key` файле настроек клиента. В случае использования HTTP интерфейса, используйте заголовок `X-ClickHouse-Quota`. + - `['client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Ключ может быть явно задан клиентом. При использовании [clickhouse-client](../../interfaces/cli.md), передайте ключевое значение в параметре `--quota_key`, или используйте параметр `quota_key` файле настроек клиента. В случае использования HTTP интерфейса, используйте заголовок `X-ClickHouse-Quota`. - `['user_name', 'client_key']` — Соединения с одинаковым ключом используют одну и ту же квоту. Если ключ не предоставлен клиентом, то квота отслеживается для `user_name`. - `['client_key', 'ip_address']` — Соединения с одинаковым ключом используют одну и ту же квоту. Если ключ не предоставлен клиентом, то квота отслеживается для `ip_address`. - `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Длины временных интервалов для расчета потребления ресурсов, в секундах. From 5e0e0a3946390736f631a94ebf5b88a0588662d6 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 16 Oct 2021 16:22:19 -0300 Subject: [PATCH 764/950] Update quotas.md --- docs/en/operations/system-tables/quotas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/quotas.md b/docs/en/operations/system-tables/quotas.md index d7a73de5046..bdcc13340f0 100644 --- a/docs/en/operations/system-tables/quotas.md +++ b/docs/en/operations/system-tables/quotas.md @@ -10,7 +10,7 @@ Columns: - `[]` — All users share the same quota. - `['user_name']` — Connections with the same user name share the same quota. - `['ip_address']` — Connections from the same IP share the same quota. - - `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota-key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header. + - `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota_key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header. - `['user_name', 'client_key']` — Connections with the same `client_key` share the same quota. If a key isn’t provided by a client, the qouta is tracked for `user_name`. - `['client_key', 'ip_address']` — Connections with the same `client_key` share the same quota. If a key isn’t provided by a client, the qouta is tracked for `ip_address`. - `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Time interval lengths in seconds. From 4390dde76cfea689602bc4c599dda626cdea3e52 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 19:48:51 +0000 Subject: [PATCH 765/950] Fix local break on timeout --- src/Client/ClientBase.cpp | 3 ++- src/Client/Connection.h | 2 ++ src/Client/IServerConnection.h | 8 ++++++++ src/Client/LocalConnection.h | 2 ++ 4 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e030d8994d5..0af952d666e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -517,6 +517,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) const size_t poll_interval = std::max(min_poll_interval, std::min(receive_timeout.totalMicroseconds(), default_poll_interval)); + bool break_on_timeout = connection->getConnectionType() != IServerConnection::Type::LOCAL; while (true) { Stopwatch receive_watch(CLOCK_MONOTONIC_COARSE); @@ -547,7 +548,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) else { double elapsed = receive_watch.elapsedSeconds(); - if (elapsed > receive_timeout.totalSeconds()) + if (break_on_timeout && elapsed > receive_timeout.totalSeconds()) { std::cout << "Timeout exceeded while receiving data from server." << " Waited for " << static_cast(elapsed) << " seconds," diff --git a/src/Client/Connection.h b/src/Client/Connection.h index b6054941aeb..848b848a104 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -60,6 +60,8 @@ public: ~Connection() override; + IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::SERVER; } + static ServerConnectionPtr createConnection(const ConnectionParameters & parameters, ContextPtr context); /// Set throttler of network traffic. One throttler could be used for multiple connections to limit total traffic. diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 42886c72182..5a853126982 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -56,6 +56,14 @@ class IServerConnection : boost::noncopyable public: virtual ~IServerConnection() = default; + enum class Type + { + SERVER, + LOCAL + }; + + virtual Type getConnectionType() const = 0; + virtual void setDefaultDatabase(const String & database) = 0; virtual void getServerVersion( diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 242d23ddc36..2bd812f200f 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -56,6 +56,8 @@ public: ~LocalConnection() override; + IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::LOCAL; } + static ServerConnectionPtr createConnection(const ConnectionParameters & connection_parameters, ContextPtr current_context, bool send_progress = false); void setDefaultDatabase(const String & database) override; From 5fc2279adf5cc901b37ccc57155857ba70d3ce9d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Oct 2021 01:23:55 +0300 Subject: [PATCH 766/950] Add RISC-V build --- cmake/linux/toolchain-riscv64.cmake | 32 +++++++++++++++++++++++++++++ contrib/sysroot | 2 +- 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 cmake/linux/toolchain-riscv64.cmake diff --git a/cmake/linux/toolchain-riscv64.cmake b/cmake/linux/toolchain-riscv64.cmake new file mode 100644 index 00000000000..1ccbd3ee0da --- /dev/null +++ b/cmake/linux/toolchain-riscv64.cmake @@ -0,0 +1,32 @@ +set (CMAKE_TRY_COMPILE_TARGET_TYPE STATIC_LIBRARY) + +set (CMAKE_SYSTEM_NAME "Linux") +set (CMAKE_SYSTEM_PROCESSOR "riscv64") +set (CMAKE_C_COMPILER_TARGET "riscv64-linux-gnu") +set (CMAKE_CXX_COMPILER_TARGET "riscv64-linux-gnu") +set (CMAKE_ASM_COMPILER_TARGET "riscv64-linux-gnu") + +set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-riscv64") + +set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}") + +find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") +find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9") + +set (CMAKE_AR "${LLVM_AR_PATH}" CACHE FILEPATH "" FORCE) +set (CMAKE_RANLIB "${LLVM_RANLIB_PATH}" CACHE FILEPATH "" FORCE) + +set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") + +set (LINKER_NAME "ld.lld" CACHE STRING "" FORCE) + +set (CMAKE_EXE_LINKER_FLAGS_INIT "-fuse-ld=lld") +set (CMAKE_SHARED_LINKER_FLAGS_INIT "-fuse-ld=lld") + +set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) + +set (HAS_POST_2038_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) +set (HAS_POST_2038_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/contrib/sysroot b/contrib/sysroot index e4663925b73..6172893931e 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit e4663925b73beb57dd29154844c8d50441146753 +Subproject commit 6172893931e19b028f9cabb7095a44361be863df From ceb98cf42ff20032fadc23698ea031aec7ab4483 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 17 Oct 2021 03:04:53 +0400 Subject: [PATCH 767/950] Add comments --- programs/install/Install.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 6b2d3a58a22..606af7ecd0d 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -77,6 +77,7 @@ namespace ErrorCodes #define END_HILITE "\033[0m" #if defined(OS_DARWIN) +/// Until createUser() and createGroup() are implemented, only sudo-less installations are supported/default for macOS. static constexpr auto DEFAULT_CLICKHOUSE_SERVER_USER = ""; static constexpr auto DEFAULT_CLICKHOUSE_SERVER_GROUP = ""; static constexpr auto DEFAULT_CLICKHOUSE_BRIDGE_USER = ""; @@ -152,6 +153,9 @@ static void createGroup(const String & group_name) if (!group_name.empty()) { #if defined(OS_DARWIN) + + // TODO: implement. + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a group in macOS"); #else std::string command = fmt::format("groupadd -r {}", group_name); @@ -166,6 +170,9 @@ static void createUser(const String & user_name, [[maybe_unused]] const String & if (!user_name.empty()) { #if defined(OS_DARWIN) + + // TODO: implement. + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unable to create a user in macOS"); #else std::string command = group_name.empty() From b034c913db468b9db95d47ea8882172e54763541 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 15 Oct 2021 14:16:41 +0800 Subject: [PATCH 768/950] update --- contrib/replxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/replxx b/contrib/replxx index 89abeea7516..b0c266c2d8a 160000 --- a/contrib/replxx +++ b/contrib/replxx @@ -1 +1 @@ -Subproject commit 89abeea7516a2a9b6aad7bfecc132f608ff14a3d +Subproject commit b0c266c2d8a835784181e17292b421848c78c6b8 From 0285a15aeca720c4ecccb16755b9d0bd81de0d45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Oct 2021 05:52:00 +0300 Subject: [PATCH 769/950] Minor changes to install script --- docs/_includes/install/universal.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index 7cba682e772..eaea4cc69ed 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -47,13 +47,17 @@ then fi URL="https://builds.clickhouse.com/master/${DIR}/clickhouse" +echo echo "Will download ${URL}" +echo curl -O "${URL}" && chmod a+x clickhouse && +echo echo "Successfully downloaded the ClickHouse binary, you can run it as: ./clickhouse" if [ "${OS}" = "Linux" ] then + echo echo "You can also install it: sudo ./clickhouse install" fi From 131aa7701738cfc057e6f452afb79a424aae7f81 Mon Sep 17 00:00:00 2001 From: WangZengrui Date: Sun, 17 Oct 2021 11:21:58 +0800 Subject: [PATCH 770/950] fix style --- src/Functions/ReplaceRegexpImpl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index b2c5470cd96..678189f8558 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -98,7 +98,7 @@ struct ReplaceRegexpImpl size_t start_pos = 0; bool is_first_match = true; bool is_start_pos_added_one = false; - + while (start_pos < static_cast(input.length())) { /// If no more replacements possible for current string @@ -117,7 +117,7 @@ struct ReplaceRegexpImpl memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, bytes_to_copy); res_offset += bytes_to_copy; start_pos += bytes_to_copy + match.length(); - + /// To avoid infinite loop. if (is_first_match && match.length() == 0 && !replace_one && input.length() > 1) { From c2faf450d129ac9a81337b96fbb819ef22edf1e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Oct 2021 06:37:51 +0300 Subject: [PATCH 771/950] Fix error --- src/IO/ReadHelpers.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index e4452a7af0a..fda8c213ebf 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -281,7 +281,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) { /// 123+ or +123+, just stop after 123 or +123. if (has_number) - return ReturnType(true); + goto end; /// No digits read yet, but we already read sign, like ++, -+. if (has_sign) @@ -300,7 +300,7 @@ ReturnType readIntTextImpl(T & x, ReadBuffer & buf) case '-': { if (has_number) - return ReturnType(true); + goto end; if (has_sign) { From 059fc1de6997ca36e8b7b009ff31a175282bd02a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 17 Oct 2021 11:42:36 +0300 Subject: [PATCH 772/950] Allow symlinks in file storage --- src/Common/filesystemHelpers.cpp | 7 +++-- src/Common/filesystemHelpers.h | 5 ++-- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 8 +---- src/Storages/StorageFile.cpp | 10 +++++-- .../02051_symlinks_to_user_files.reference | 1 + .../02051_symlinks_to_user_files.sh | 30 +++++++++++++++++++ 7 files changed, 47 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/02051_symlinks_to_user_files.reference create mode 100755 tests/queries/0_stateless/02051_symlinks_to_user_files.sh diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 89214ad496e..f9fe8c97a14 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -118,7 +118,7 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p return absolute_path.starts_with(absolute_prefix_path); } -bool symlinkStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path) +bool fileOrSymlinkPathStartsWith(const std::filesystem::path & path, const std::filesystem::path & prefix_path) { /// Differs from pathStartsWith in how `path` is normalized before comparison. /// Make `path` absolute if it was relative and put it into normalized form: remove @@ -140,13 +140,14 @@ bool pathStartsWith(const String & path, const String & prefix_path) return pathStartsWith(filesystem_path, filesystem_prefix_path); } -bool symlinkStartsWith(const String & path, const String & prefix_path) +bool fileOrSymlinkPathStartsWith(const String & path, const String & prefix_path) { auto filesystem_path = std::filesystem::path(path); auto filesystem_prefix_path = std::filesystem::path(prefix_path); - return symlinkStartsWith(filesystem_path, filesystem_prefix_path); + return fileOrSymlinkPathStartsWith(filesystem_path, filesystem_prefix_path); } + } diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index de5802cde6d..f626198920e 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -35,8 +35,9 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p /// Returns true if path starts with prefix path bool pathStartsWith(const String & path, const String & prefix_path); -/// Returns true if symlink starts with prefix path -bool symlinkStartsWith(const String & path, const String & prefix_path); +/// Same as pathStartsWith, but wihtout canonization, i.e. allowed to check symlinks. +/// (Path is made absolute and normalized) +bool fileOrSymlinkPathStartsWith(const String & path, const String & prefix_path); } diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 8c1f099f344..7fd2dbf80f1 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -31,7 +31,7 @@ FileDictionarySource::FileDictionarySource( , context(context_) { auto user_files_path = context->getUserFilesPath(); - if (created_from_ddl && !pathStartsWith(filepath, user_files_path)) + if (created_from_ddl && !fileOrSymlinkPathStartsWith(filepath, user_files_path)) throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", filepath, user_files_path); } diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index f117cfb179e..42683fb884c 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -41,13 +41,7 @@ LibraryDictionarySource::LibraryDictionarySource( , context(Context::createCopy(context_)) { auto dictionaries_lib_path = context->getDictionariesLibPath(); - bool path_checked = false; - if (fs::is_symlink(path)) - path_checked = symlinkStartsWith(path, dictionaries_lib_path); - else - path_checked = pathStartsWith(path, dictionaries_lib_path); - - if (created_from_ddl && !path_checked) + if (created_from_ddl && !fileOrSymlinkPathStartsWith(path, dictionaries_lib_path)) throw Exception(ErrorCodes::PATH_ACCESS_DENIED, "File path {} is not inside {}", path, dictionaries_lib_path); if (!fs::exists(path)) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 4ae55272db6..24377017987 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -124,8 +125,8 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di return; /// "/dev/null" is allowed for perf testing - if (!startsWith(table_path, db_dir_path) && table_path != "/dev/null") - throw Exception("File is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED); + if (!fileOrSymlinkPathStartsWith(table_path, db_dir_path) && table_path != "/dev/null") + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "File `{}` is not inside `{}`", table_path, db_dir_path); if (fs::exists(table_path) && fs::is_directory(table_path)) throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); @@ -140,7 +141,10 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user fs_table_path = user_files_absolute_path / fs_table_path; Strings paths; - const String path = fs::weakly_canonical(fs_table_path); + /// Do not use fs::canonical or fs::weakly_canonical. + /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. + String path = std::filesystem::absolute(fs_table_path); + path = fs::path(path).lexically_normal(); /// Normalize path. if (path.find_first_of("*?{") == std::string::npos) { std::error_code error; diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.reference b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh new file mode 100755 index 00000000000..3a9882a441c --- /dev/null +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# See 01658_read_file_to_string_column.sh +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') + +mkdir -p "${user_files_path}/" +chmod 777 "${user_files_path}" + +export FILE="test_symlink_${CLICKHOUSE_DATABASE}" + +symlink_path=${user_files_path}/${FILE} +file_path=$CUR_DIR/${FILE} + +function cleanup() +{ + rm ${symlink_path} ${file_path} +} +trap cleanup EXIT + +touch ${file_path} +ln -s ${file_path} ${symlink_path} + +${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; +${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String')"; + From cd22ca616752c07f1808eeb676c9747693e88a1e Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 17 Oct 2021 13:13:47 +0300 Subject: [PATCH 773/950] Update filesystemHelpers.h --- src/Common/filesystemHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index f626198920e..2b805ce0c68 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -35,7 +35,7 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p /// Returns true if path starts with prefix path bool pathStartsWith(const String & path, const String & prefix_path); -/// Same as pathStartsWith, but wihtout canonization, i.e. allowed to check symlinks. +/// Same as pathStartsWith, but without canonization, i.e. allowed to check symlinks. /// (Path is made absolute and normalized) bool fileOrSymlinkPathStartsWith(const String & path, const String & prefix_path); From 07b44713b63b0e5292987f4937435830076dd203 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 17 Oct 2021 13:45:32 +0300 Subject: [PATCH 774/950] Ping CI --- src/Common/filesystemHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 2b805ce0c68..fc3a4f15573 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -36,7 +36,7 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool pathStartsWith(const String & path, const String & prefix_path); /// Same as pathStartsWith, but without canonization, i.e. allowed to check symlinks. -/// (Path is made absolute and normalized) +/// (Path is made absolute and normalized.) bool fileOrSymlinkPathStartsWith(const String & path, const String & prefix_path); } From 5633865df159132e65242d9ce09f9f0206174ed8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 17 Oct 2021 14:16:40 +0300 Subject: [PATCH 775/950] Update src/Storages/StorageFile.cpp Co-authored-by: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> --- src/Storages/StorageFile.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 24377017987..bdc0c203d59 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -143,7 +143,7 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user Strings paths; /// Do not use fs::canonical or fs::weakly_canonical. /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. - String path = std::filesystem::absolute(fs_table_path); + String path = fs::absolute(fs_table_path); path = fs::path(path).lexically_normal(); /// Normalize path. if (path.find_first_of("*?{") == std::string::npos) { From 9378b93e4154ac417f1f2f35fefec857ebb76891 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 17 Oct 2021 14:49:27 +0000 Subject: [PATCH 776/950] fix --- src/Common/ErrorCodes.cpp | 2 +- src/Common/filesystemHelpers.cpp | 2 +- src/Common/filesystemHelpers.h | 3 +- src/Dictionaries/LibraryDictionarySource.cpp | 2 +- src/Storages/FileLog/DirectoryWatcherBase.cpp | 32 +++- src/Storages/FileLog/DirectoryWatcherBase.h | 6 +- .../FileLog/FileLogDirectoryWatcher.cpp | 30 ++-- .../FileLog/FileLogDirectoryWatcher.h | 16 +- src/Storages/FileLog/FileLogSettings.h | 7 +- src/Storages/FileLog/FileLogSource.cpp | 9 +- src/Storages/FileLog/FileLogSource.h | 2 +- .../FileLog/ReadBufferFromFileLog.cpp | 11 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 3 +- src/Storages/FileLog/StorageFileLog.cpp | 152 ++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 22 ++- src/Storages/IStorage.h | 3 +- 16 files changed, 189 insertions(+), 113 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 05d4a5cbc5a..e62161eca6d 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,7 +589,7 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(621, CANNOT_NORMALIZE_STRING) \ - M(622, NOT_REGULAR_FILE) \ + M(622, BAD_FILE_TYPE) \ M(623, IO_SETUP_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 89214ad496e..4c9cd37fc27 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -140,7 +140,7 @@ bool pathStartsWith(const String & path, const String & prefix_path) return pathStartsWith(filesystem_path, filesystem_prefix_path); } -bool symlinkStartsWith(const String & path, const String & prefix_path) +bool isPathOrSymlinkStartsWith(const String & path, const String & prefix_path) { auto filesystem_path = std::filesystem::path(path); auto filesystem_prefix_path = std::filesystem::path(prefix_path); diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index de5802cde6d..104d4cd8987 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -36,8 +36,7 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool pathStartsWith(const String & path, const String & prefix_path); /// Returns true if symlink starts with prefix path -bool symlinkStartsWith(const String & path, const String & prefix_path); - +bool isPathOrSymlinkStartsWith(const String & path, const String & prefix_path); } namespace FS diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 551bb1ee2dd..57b1ac852b9 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -44,7 +44,7 @@ LibraryDictionarySource::LibraryDictionarySource( auto dictionaries_lib_path = context->getDictionariesLibPath(); bool path_checked = false; if (fs::is_symlink(path)) - path_checked = symlinkStartsWith(path, dictionaries_lib_path); + path_checked = isPathOrSymlinkStartsWith(path, dictionaries_lib_path); else path_checked = pathStartsWith(path, dictionaries_lib_path); diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 9ea2e7e1824..cad2072a8e6 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; - extern const int DIRECTORY_DOESNT_EXIST; + extern const int BAD_FILE_TYPE; extern const int IO_SETUP_ERROR; } @@ -20,19 +20,19 @@ static constexpr int buffer_size = 4096; DirectoryWatcherBase::DirectoryWatcherBase( FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) - : WithContext(context_->getGlobalContext()), owner(owner_), path(path_), event_mask(event_mask_) + : WithContext(context_), owner(owner_), path(path_), event_mask(event_mask_) { if (!std::filesystem::exists(path)) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist", path); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Path {} does not exist", path); if (!std::filesystem::is_directory(path)) - throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory", path); + throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path {} is not a directory", path); fd = inotify_init(); if (fd == -1) throw Exception("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); - watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); + watch_task = getContext()->getSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); } @@ -63,6 +63,7 @@ void DirectoryWatcherBase::watchFunc() pfd.events = POLLIN; while (!stopped) { + const auto & settings = owner.storage.getFileLogSettings(); if (poll(&pfd, 1, 500) > 0 && pfd.revents & POLLIN) { int n = read(fd, buffer.data(), buffer.size()); @@ -106,8 +107,25 @@ void DirectoryWatcherBase::watchFunc() n -= sizeof(inotify_event) + p_event->len; } } + + /// Wake up reader thread + auto & mutex = owner.storage.getMutex(); + auto & cv = owner.storage.getConditionVariable(); + std::unique_lock lock(mutex); + owner.storage.setNewEvents(); + lock.unlock(); + cv.notify_one(); + } + else + { + if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; + break; } } + + if (!stopped) + watch_task->scheduleAfter(milliseconds_to_wait); } @@ -115,6 +133,9 @@ DirectoryWatcherBase::~DirectoryWatcherBase() { stop(); close(fd); + + if (watch_task) + watch_task->deactivate(); } void DirectoryWatcherBase::start() @@ -123,7 +144,6 @@ void DirectoryWatcherBase::start() watch_task->activateAndSchedule(); } - void DirectoryWatcherBase::stop() { stopped = true; diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index db89746358b..88f864ac17d 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -12,8 +12,10 @@ class FileLogDirectoryWatcher; class DirectoryWatcherBase : WithContext { + /// Most of code in this class is copy from the Poco project: + /// https://github.com/ClickHouse-Extras/poco/blob/clickhouse/Foundation/src/DirectoryWatcher.cpp /// This class is used to get notifications about changes - /// to the filesystem, more specifically, to a specific + /// to the filesystem, more precisely, to a specific /// directory. Changes to a directory are reported via /// events. /// @@ -95,6 +97,8 @@ private: std::atomic stopped{false}; + uint64_t milliseconds_to_wait; + const std::string path; int event_mask; int fd; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 4dadbed3f09..c4a153dbfe7 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -2,8 +2,9 @@ namespace DB { -FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, StorageFileLog & storage_, ContextPtr context_) : path(path_) + , storage(storage_) , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) , dw(std::make_unique(*this, path, context_)) { @@ -39,11 +40,11 @@ void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent e if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } @@ -57,11 +58,11 @@ void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } @@ -80,14 +81,17 @@ void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEven /// Already have MODIFY event for this file if (auto it = events.find(event_path); it != events.end()) { - if (it->second.back().type == ev.event) + if (it->second.received_modification_event) return; else - it->second.emplace_back(info); + { + it->second.received_modification_event = true; + it->second.file_events.emplace_back(info); + } } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.received_modification_event = true, .file_events = std::vector{info}}); } } @@ -100,11 +104,11 @@ void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEve if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } @@ -117,18 +121,18 @@ void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } void FileLogDirectoryWatcher::onError(Exception e) { std::lock_guard lock(mutex); - LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); + LOG_ERROR(log, "Error happened during watching directory: {}", error.error_msg); error.has_error = true; error.error_msg = e.message(); } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 4259cc8722e..8e0ecf5358c 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -19,7 +20,13 @@ public: std::string callback; }; - using Events = std::unordered_map>; + struct FileEvents + { + bool received_modification_event = false; + std::vector file_events; + }; + + using Events = std::unordered_map; struct Error { @@ -27,7 +34,7 @@ public: std::string error_msg = {}; }; - explicit FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_); + FileLogDirectoryWatcher(const std::string & path_, StorageFileLog & storage_, ContextPtr context_); ~FileLogDirectoryWatcher() = default; Events getEventsAndReset(); @@ -36,6 +43,8 @@ public: const std::string & getPath() const; +private: + friend class DirectoryWatcherBase; /// Here must pass by value, otherwise will lead to stack-use-of-scope void onItemAdded(DirectoryWatcherBase::DirectoryEvent ev); void onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev); @@ -44,9 +53,10 @@ public: void onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev); void onError(Exception); -private: const std::string path; + StorageFileLog & storage; + /// Note, in order to avoid data race found by fuzzer, put events before dw, /// such that when this class destruction, dw will be destructed before events. /// The data race is because dw create a separate thread to monitor file events diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index 5ed55d94426..d14120d0ba0 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -14,7 +14,10 @@ class ASTStorage; M(Milliseconds, poll_timeout_ms, 0, "Timeout for single poll from StorageFileLog.", 0) \ M(UInt64, poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single StorageFileLog poll.", 0) \ M(UInt64, max_block_size, 0, "Number of row collected by poll(s) for flushing data from StorageFileLog.", 0) \ - M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0) + M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0) \ + M(Milliseconds, poll_directory_watch_events_backoff_init, 500, "The initial sleep value for watch directory thread.", 0) \ + M(Milliseconds, poll_directory_watch_events_backoff_max, 32000, "The max sleep value for watch directory thread.", 0) \ + M(UInt64, poll_directory_watch_events_backoff_factor, 2, "The speed of backoff, exponential by default", 0) #define LIST_OF_FILELOG_SETTINGS(M) \ FILELOG_RELATED_SETTINGS(M) \ @@ -23,7 +26,7 @@ class ASTStorage; DECLARE_SETTINGS_TRAITS(FileLogSettingsTraits, LIST_OF_FILELOG_SETTINGS) -/** Settings for the Kafka engine. +/** Settings for the FileLog engine. * Could be loaded from a CREATE TABLE query (SETTINGS clause). */ struct FileLogSettings : public BaseSettings diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 7beee42c99c..bb0fcc90016 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -47,16 +47,19 @@ FileLogSource::FileLogSource( void FileLogSource::onFinish() { storage.closeFilesAndStoreMeta(start, end); + finished = true; } Chunk FileLogSource::generate() { + /// Store metas of last written chunk into disk + storage.storeMetas(start, end); + if (!buffer || buffer->noRecords()) { /// There is no onFinish for ISource, we call it /// when no records return to close files onFinish(); - finished = true; return {}; } @@ -104,7 +107,6 @@ Chunk FileLogSource::generate() if (total_rows == 0) { onFinish(); - finished = true; return {}; } @@ -122,9 +124,6 @@ Chunk FileLogSource::generate() auto converting_actions = std::make_shared(std::move(converting_dag)); converting_actions->execute(result_block); - /// After generate each block, store metas into disk - storage.storeMetas(start, end); - return Chunk(result_block.getColumns(), result_block.rows()); } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index cdf60cd4fc1..b8b579beaed 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -58,7 +58,7 @@ private: Block virtual_header; /// The start pos and end pos of files responsible by this stream, - /// does not includind end + /// does not include end size_t start; size_t end; }; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index f935decd7f8..384de64cd3f 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -47,7 +47,7 @@ bool ReadBufferFromFileLog::poll() if (new_records.empty()) { buffer_status = BufferStatus::NO_RECORD_RETURNED; - LOG_TRACE(log, "No records returned"); + LOG_TRACE(log, "No new records to read"); return false; } else @@ -107,16 +107,19 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); if (!file_ctx.reader) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized", file_meta.file_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} is not initialized", file_meta.file_name); auto & reader = file_ctx.reader.value(); StorageFileLog::assertStreamGood(reader); Record record; - while (read_records_size < need_records_size && static_cast(reader.tellg()) < file_meta.last_open_end) + while (read_records_size < need_records_size) { /// Need to get offset before reading record from stream - record.offset = reader.tellg(); + auto offset = reader.tellg(); + if (static_cast(offset) < file_meta.last_open_end) + break; + record.offset = offset; StorageFileLog::assertStreamGood(reader); record.file_name = file_name; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index b4102870c3a..117a858de3b 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -71,6 +71,7 @@ private: { RecordData data; std::string file_name; + /// Offset is the start of a row, which is needed for virtual columns. UInt64 offset; }; using Records = std::vector; @@ -78,7 +79,7 @@ private: Records records; Records::const_iterator current; - String current_file = {}; + String current_file; UInt64 current_offset = 0; using TaskThread = BackgroundSchedulePool::TaskHolder; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 516be811666..66add8128ef 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -37,11 +37,12 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int CANNOT_STAT; - extern const int NOT_REGULAR_FILE; + extern const int BAD_FILE_TYPE; extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; extern const int TABLE_METADATA_ALREADY_EXISTS; extern const int CANNOT_SELECT; + extern const int QUERY_NOT_ALLOWED; } namespace @@ -80,44 +81,31 @@ StorageFileLog::StorageFileLog( loadMetaFiles(attach); loadFiles(); -#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); -#endif if (path_is_directory) - directory_watch = std::make_unique(root_data_path, context_); + directory_watch = std::make_unique(root_data_path, *this, getContext()); - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + auto thread = getContext()->getSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); } catch (...) { + if (!attach) + throw; tryLogCurrentException(__PRETTY_FUNCTION__); } } void StorageFileLog::loadMetaFiles(bool attach) { - /// We just use default storage policy - auto storage_policy = getContext()->getStoragePolicy("default"); - auto data_volume = storage_policy->getVolume(0); - root_meta_path = std::filesystem::path(data_volume->getDisk()->getPath()) / getStorageID().getTableName(); + const auto & storage_id = getStorageID(); + root_meta_path = std::filesystem::path(getContext()->getPath()) / "metadata" / "filelog_storage_metadata" / storage_id.getDatabaseName() + / storage_id.getTableName(); - /// Create table, just create meta data directory - if (!attach) - { - if (std::filesystem::exists(root_meta_path)) - { - throw Exception( - ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, - "Metadata files already exist by path: {}, remove them manually if it is intended", - root_meta_path); - } - std::filesystem::create_directories(root_meta_path); - } /// Attach table - else + if (attach) { /// Meta file may lost, log and create directory if (!std::filesystem::exists(root_meta_path)) @@ -128,14 +116,26 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Load all meta info to file_infos; deserialize(); } + /// Create table, just create meta data directory + else + { + if (std::filesystem::exists(root_meta_path)) + { + throw Exception( + ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, + "Metadata files already exist by path: {}, remove them manually if it is intended", + root_meta_path); + } + std::filesystem::create_directories(root_meta_path); + } } void StorageFileLog::loadFiles() { - if (!symlinkStartsWith(path, getContext()->getUserFilesPath())) + if (!isPathOrSymlinkStartsWith(path, getContext()->getUserFilesPath())) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, "The absolute data path should start with user_files_path {}", getContext()->getUserFilesPath()); + ErrorCodes::BAD_ARGUMENTS, "The absolute data path should be inside `user_files_path`({})", getContext()->getUserFilesPath()); } auto absolute_path = std::filesystem::absolute(path); @@ -181,10 +181,7 @@ void StorageFileLog::loadFiles() if (it->second.file_name != file) { it->second.file_name = file; - if (std::filesystem::exists(getFullMetaPath(it->second.file_name))) - { - std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); - } + std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); } } /// New file @@ -264,12 +261,14 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const void StorageFileLog::deserialize() { + /// In case of single file (not a watched directory), + /// iterated directoy always has one file inside. for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) { if (!dir_entry.is_regular_file()) { throw Exception( - ErrorCodes::NOT_REGULAR_FILE, + ErrorCodes::BAD_FILE_TYPE, "The file {} under {} is not a regular file when deserializing meta files", dir_entry.path().c_str(), root_meta_path); @@ -315,15 +314,14 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - auto table_id = getStorageID(); - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); /// If there are MVs depended on this table, we just forbid reading - if (dependencies_count) + if (has_dependent_mv) { throw Exception( - ErrorCodes::CANNOT_READ_ALL_DATA, - "Can not read from table {}, because it has been depended by other tables", - table_id.getTableName()); + ErrorCodes::QUERY_NOT_ALLOWED, + "Can not make `SELECT` query from table {}, because it has attached dependencies. Remove dependant materialized views if " + "needed", + getStorageID().getTableName()); } if (running_streams.load(std::memory_order_relaxed)) @@ -331,16 +329,12 @@ Pipe StorageFileLog::read( throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } - /// We need this lock, in case read and streamToViews execute at the same time. - /// In case of MV attached during reading - std::lock_guard lock(status_mutex); - updateFileInfos(); /// No files to parse if (file_infos.file_names.empty()) { - LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); + LOG_WARNING(log, "There is a idle table named {}, no files need to parse.", getName()); return Pipe{}; } @@ -371,12 +365,12 @@ Pipe StorageFileLog::read( void StorageFileLog::increaseStreams() { - running_streams.fetch_add(1, std::memory_order_relaxed); + running_streams += 1; } void StorageFileLog::reduceStreams() { - running_streams.fetch_sub(1, std::memory_order_relaxed); + running_streams -= 1; } void StorageFileLog::drop() @@ -457,9 +451,14 @@ void StorageFileLog::openFilesAndSetPos() auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken", file); + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Last saved offsset for File {} is bigger than file size ({} > {})", + file, + meta.last_writen_position, + file_end); } - /// update file end at the monment, used in ReadBuffer and serialize + /// update file end at the moment, used in ReadBuffer and serialize meta.last_open_end = file_end; reader.seekg(meta.last_writen_position); @@ -471,11 +470,9 @@ void StorageFileLog::openFilesAndSetPos() void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { -#ifndef NDEBUG assert(start >= 0); assert(start < end); assert(end <= file_infos.file_names.size()); -#endif for (size_t i = start; i < end; ++i) { @@ -494,11 +491,9 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) void StorageFileLog::storeMetas(size_t start, size_t end) { -#ifndef NDEBUG assert(start >= 0); assert(start < end); assert(end <= file_infos.file_names.size()); -#endif for (size_t i = start; i < end; ++i) { @@ -577,9 +572,11 @@ void StorageFileLog::threadFunc() { try { + updateFileInfos(); auto table_id = getStorageID(); // Check if at least one direct dependency is attached size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + if (dependencies_count) { auto start_time = std::chrono::steady_clock::now(); @@ -595,13 +592,15 @@ void StorageFileLog::threadFunc() if (streamToViews()) { LOG_TRACE(log, "Stream stalled. Reschedule."); - if (milliseconds_to_wait < BACKOFF_TRESHOLD) - milliseconds_to_wait *= 2; + if (!path_is_directory + && milliseconds_to_wait + < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= filelog_settings->poll_directory_watch_events_backoff_factor.value; break; } else { - milliseconds_to_wait = RESCHEDULE_MS; + milliseconds_to_wait = filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); } auto ts = std::chrono::steady_clock::now(); @@ -611,7 +610,6 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } - updateFileInfos(); } } } @@ -622,12 +620,27 @@ void StorageFileLog::threadFunc() // Wait for attached views if (!task->stream_cancelled) - task->holder->scheduleAfter(milliseconds_to_wait); + { + if (path_is_directory) + { + std::unique_lock lock(mutex); + /// Waiting for watch directory thread to wake up + cv.wait(lock, [this] { return has_new_events; }); + has_new_events = false; + task->holder->schedule(); + } + else + task->holder->scheduleAfter(milliseconds_to_wait); + } } bool StorageFileLog::streamToViews() { - std::lock_guard lock(status_mutex); + if (running_streams.load(std::memory_order_relaxed)) + { + throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); + } + has_dependent_mv = true; Stopwatch watch; auto table_id = getStorageID(); @@ -726,6 +739,19 @@ void registerStorageFileLog(StorageFactory & factory) throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } + size_t init_sleep_time = filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); + size_t max_sleep_time = filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds(); + if (init_sleep_time > max_sleep_time) + { + throw Exception( + "poll_directory_watch_events_backoff_init can not be greater than poll_directory_watch_events_backoff_max", + ErrorCodes::BAD_ARGUMENTS); + } + + if (filelog_settings->poll_directory_watch_events_backoff_factor.changed + && !filelog_settings->poll_directory_watch_events_backoff_factor.value) + throw Exception("poll_directory_watch_events_backoff_factor can not be 0", ErrorCodes::BAD_ARGUMENTS); + if (args_count != 2) throw Exception( "Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -763,11 +789,9 @@ bool StorageFileLog::updateFileInfos() /// For table just watch one file, we can not use directory monitor to watch it if (!path_is_directory) { -#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); assert(file_infos.file_names.size() == 1); -#endif if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end()) { @@ -783,18 +807,16 @@ bool StorageFileLog::updateFileInfos() if (error.has_error) LOG_ERROR(log, "Error happened during watching directory {}: {}", directory_watch->getPath(), error.error_msg); -/// These file infos should always have same size(one for one) before update and after update -#ifndef NDEBUG + /// These file infos should always have same size(one for one) before update and after update assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); -#endif auto events = directory_watch->getEventsAndReset(); - for (const auto & [file_name, event_infos] : events) + for (const auto & [file_name, event_info] : events) { String file_path = getFullDataPath(file_name); - for (const auto & event_info : event_infos) + for(const auto & event_info : event_info.file_events) { switch (event_info.type) { @@ -903,22 +925,20 @@ bool StorageFileLog::updateFileInfos() } file_infos.file_names.swap(valid_files); -/// These file infos should always have same size(one for one) -#ifndef NDEBUG + /// These file infos should always have same size(one for one) assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); -#endif return events.empty() || file_infos.file_names.empty(); } NamesAndTypesList StorageFileLog::getVirtuals() const { - return NamesAndTypesList{{"_file_name", std::make_shared()}, {"_offset", std::make_shared()}}; + return NamesAndTypesList{{"_filename", std::make_shared()}, {"_offset", std::make_shared()}}; } Names StorageFileLog::getVirtualColumnNames() { - return {"_file_name", "_offset"}; + return {"_filename", "_offset"}; } } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 1b1d5f6671a..04c83d8837d 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -11,6 +10,7 @@ #include #include +#include #include #include #include @@ -23,6 +23,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class FileLogDirectoryWatcher; + class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; @@ -75,8 +77,8 @@ public: struct FileMeta { String file_name; - UInt64 last_writen_position{}; - UInt64 last_open_end{}; + UInt64 last_writen_position = 0; + UInt64 last_open_end = 0; }; using InodeToFileMeta = std::unordered_map; @@ -124,6 +126,12 @@ public: void increaseStreams(); void reduceStreams(); + auto & getConditionVariable() { return cv; } + auto & getMutex() { return mutex; } + void setNewEvents() { has_new_events = true; } + + const auto & getFileLogSettings() const { return filelog_settings; } + protected: StorageFileLog( const StorageID & table_id_, @@ -155,8 +163,6 @@ private: const String format_name; Poco::Logger * log; - std::mutex status_mutex; - std::unique_ptr directory_watch = nullptr; uint64_t milliseconds_to_wait; @@ -177,6 +183,12 @@ private: /// later select should forbid to execute. std::atomic running_streams = 0; + std::atomic has_dependent_mv = false; + + std::mutex mutex; + bool has_new_events = false; + std::condition_variable cv; + void loadFiles(); void loadMetaFiles(bool attach); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4de7cbc71ff..419c1bf173c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -586,7 +586,8 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } - /// Do we should call table->drop immediately when drop table + /// Should table->drop be called at once or with delay (in case of atomic database engine). + /// Needed for integration engines, when there must be no delay for calling drop() method. virtual bool dropTableImmediately() { return false; } private: From 5dab184d8b0c2ef1e59d5e194b21316750598e04 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 17 Oct 2021 18:21:43 +0300 Subject: [PATCH 777/950] Update 02051_symlinks_to_user_files.sh --- tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index 3a9882a441c..7d1fffba74d 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -16,6 +16,8 @@ export FILE="test_symlink_${CLICKHOUSE_DATABASE}" symlink_path=${user_files_path}/${FILE} file_path=$CUR_DIR/${FILE} +chmod +w ${file_path} + function cleanup() { rm ${symlink_path} ${file_path} From 301caa80918f36ce32139d8e6554e314ba494183 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Oct 2021 20:52:21 +0300 Subject: [PATCH 778/950] Update test --- .../0_stateless/01425_decimal_parse_big_negative_exponent.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql b/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql index 7f276d1f8d4..1387206b882 100644 --- a/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql +++ b/tests/queries/0_stateless/01425_decimal_parse_big_negative_exponent.sql @@ -1,4 +1,4 @@ -SELECT '-1E9-1E9-1E9-1E9' AS x, toDecimal32(x, 0); -- { serverError 6 } +SELECT '-1E9-1E9-1E9-1E9' AS x, toDecimal32(x, 0); -- { serverError 69 } SELECT '-1E9' AS x, toDecimal32(x, 0); -- { serverError 69 } SELECT '1E-9' AS x, toDecimal32(x, 0); SELECT '1E-8' AS x, toDecimal32(x, 0); From a8a7ba90056d09dfaa7ab717a992f0535fcddc00 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 17 Oct 2021 22:09:17 +0300 Subject: [PATCH 779/950] Update 02051_symlinks_to_user_files.sh --- tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index 7d1fffba74d..53c50542b06 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -16,7 +16,9 @@ export FILE="test_symlink_${CLICKHOUSE_DATABASE}" symlink_path=${user_files_path}/${FILE} file_path=$CUR_DIR/${FILE} +touch ${file_path} chmod +w ${file_path} +ln -s ${file_path} ${symlink_path} function cleanup() { @@ -24,9 +26,6 @@ function cleanup() } trap cleanup EXIT -touch ${file_path} -ln -s ${file_path} ${symlink_path} - ${CLICKHOUSE_CLIENT} --query="insert into table function file('${symlink_path}', 'Values', 'a String') select 'OK'"; ${CLICKHOUSE_CLIENT} --query="select * from file('${symlink_path}', 'Values', 'a String')"; From eb66442feb7429c7f09f1125f2b82b3f1fc7376d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 17 Oct 2021 16:21:21 -0300 Subject: [PATCH 780/950] Update external-dicts-dict-layout.md --- .../external-dictionaries/external-dicts-dict-layout.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 30e050ef9ef..c4a359efca4 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -422,7 +422,7 @@ Similar to `cache`, but stores data on SSD and index in RAM. All cache dictionar 1048576 - /var/lib/clickhouse/clickhouse_dictionaries/test_dict + /var/lib/clickhouse/user_files/test_dict ``` @@ -431,7 +431,7 @@ or ``` sql LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 - PATH ./user_files/test_dict)) + PATH '/var/lib/clickhouse/user_files/test_dict')) ``` ### complex_key_ssd_cache {#complex-key-ssd-cache} From 084938fa8c6c1ee7c5c026e8f2b2dbe912fdc6c0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 17 Oct 2021 16:28:22 -0300 Subject: [PATCH 781/950] Update external-dicts-dict-layout.md --- .../external-dictionaries/external-dicts-dict-layout.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 06fe4ae327a..73736344c57 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -352,7 +352,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) 1048576 - /var/lib/clickhouse/clickhouse_dictionaries/test_dict + /var/lib/clickhouse/user_files/test_dict ``` @@ -361,7 +361,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) ``` sql LAYOUT(SSD_CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 - PATH ./user_files/test_dict)) + PATH '/var/lib/clickhouse/user_files/test_dict')) ``` ### complex_key_ssd_cache {#complex-key-ssd-cache} From 9de534f96c751d524f96961d620ae043618e3cdc Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 17 Oct 2021 23:10:01 +0300 Subject: [PATCH 782/950] Revert special contribs and set folder manually for them until fixed --- cmake/find/cxx.cmake | 2 ++ cmake/find/unwind.cmake | 1 + contrib/CMakeLists.txt | 9 --------- contrib/libcxx-cmake/CMakeLists.txt | 1 + contrib/libcxxabi-cmake/CMakeLists.txt | 1 + contrib/libunwind-cmake/CMakeLists.txt | 1 + 6 files changed, 6 insertions(+), 9 deletions(-) diff --git a/cmake/find/cxx.cmake b/cmake/find/cxx.cmake index b96ba1e1b65..b1da125e219 100644 --- a/cmake/find/cxx.cmake +++ b/cmake/find/cxx.cmake @@ -50,6 +50,8 @@ endif () if (NOT HAVE_LIBCXX AND NOT MISSING_INTERNAL_LIBCXX_LIBRARY) set (LIBCXX_LIBRARY cxx) set (LIBCXXABI_LIBRARY cxxabi) + add_subdirectory(contrib/libcxxabi-cmake) + add_subdirectory(contrib/libcxx-cmake) # Exception handling library is embedded into libcxxabi. diff --git a/cmake/find/unwind.cmake b/cmake/find/unwind.cmake index 9ae23ae23c7..c9f5f30a5d6 100644 --- a/cmake/find/unwind.cmake +++ b/cmake/find/unwind.cmake @@ -1,6 +1,7 @@ option (USE_UNWIND "Enable libunwind (better stacktraces)" ${ENABLE_LIBRARIES}) if (USE_UNWIND) + add_subdirectory(contrib/libunwind-cmake) set (UNWIND_LIBRARIES unwind) set (EXCEPTION_HANDLING_LIBRARY ${UNWIND_LIBRARIES}) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1be61db40db..676654452de 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -21,15 +21,6 @@ endif() set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1) -if (USE_INTERNAL_LIBCXX_LIBRARY AND NOT MISSING_INTERNAL_LIBCXX_LIBRARY) - add_subdirectory(libcxxabi-cmake) - add_subdirectory(libcxx-cmake) -endif () - -if (USE_UNWIND) - add_subdirectory(libunwind-cmake) -endif () - add_subdirectory (abseil-cpp-cmake) add_subdirectory (magic-enum-cmake) add_subdirectory (boost-cmake) diff --git a/contrib/libcxx-cmake/CMakeLists.txt b/contrib/libcxx-cmake/CMakeLists.txt index ac67f2563a3..2ec6dbff1a1 100644 --- a/contrib/libcxx-cmake/CMakeLists.txt +++ b/contrib/libcxx-cmake/CMakeLists.txt @@ -47,6 +47,7 @@ set(SRCS ) add_library(cxx ${SRCS}) +set_target_properties(cxx PROPERTIES FOLDER "contrib/libcxx-cmake") target_include_directories(cxx SYSTEM BEFORE PUBLIC $) target_compile_definitions(cxx PRIVATE -D_LIBCPP_BUILDING_LIBRARY -DLIBCXX_BUILDING_LIBCXXABI) diff --git a/contrib/libcxxabi-cmake/CMakeLists.txt b/contrib/libcxxabi-cmake/CMakeLists.txt index 0bb5d663633..425111d9b26 100644 --- a/contrib/libcxxabi-cmake/CMakeLists.txt +++ b/contrib/libcxxabi-cmake/CMakeLists.txt @@ -22,6 +22,7 @@ set(SRCS ) add_library(cxxabi ${SRCS}) +set_target_properties(cxxabi PROPERTIES FOLDER "contrib/libcxxabi-cmake") # Third party library may have substandard code. target_compile_options(cxxabi PRIVATE -w) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 1a9f5e50abd..155853a0bca 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -39,6 +39,7 @@ set(LIBUNWIND_SOURCES ${LIBUNWIND_ASM_SOURCES}) add_library(unwind ${LIBUNWIND_SOURCES}) +set_target_properties(unwind PROPERTIES FOLDER "contrib/libunwind-cmake") target_include_directories(unwind SYSTEM BEFORE PUBLIC $) target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1 -D_DEBUG -D_LIBUNWIND_IS_NATIVE_ONLY) From 34810456e1d10510c33241f9db5178914b40edd7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:07:48 +0300 Subject: [PATCH 783/950] Fix alignment for prefetch in AsynchronousReadBufferFromFileDescriptor --- src/IO/AsynchronousReadBufferFromFileDescriptor.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index c64341089d0..50d8f5819fe 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -40,6 +40,7 @@ public: : ReadBufferFromFileBase(buf_size, existing_memory, alignment), reader(std::move(reader_)), priority(priority_), required_alignment(alignment), fd(fd_) { + prefetch_buffer.alignment = alignment; } ~AsynchronousReadBufferFromFileDescriptor() override; From 9cc1178ebc11f0f44e60670762306a4a3f854b0a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:07:48 +0300 Subject: [PATCH 784/950] BufferWithOwnMemory: do not try to align if buffer already aligned --- src/IO/BufferWithOwnMemory.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 1d9267a8518..fe2aa29f46f 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -101,6 +101,9 @@ private: if (!alignment) return value; + if (!(value % alignment)) + return value; + return (value + alignment - 1) / alignment * alignment; } From 348033d8bd8abdb32585c392d7e9f8af98c897ab Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:07:49 +0300 Subject: [PATCH 785/950] BufferWithOwnMemory: make size aligned not capacity This will fix pread_fake_async/pread_threadpool with min_bytes_to_use_direct_io --- src/IO/BufferWithOwnMemory.h | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index fe2aa29f46f..17bd0df9993 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -88,7 +88,7 @@ struct Memory : boost::noncopyable, Allocator } else { - size_t new_capacity = align(new_size + pad_right, alignment); + size_t new_capacity = align(new_size, alignment) + pad_right; m_data = static_cast(Allocator::realloc(m_data, m_capacity, new_capacity, alignment)); m_capacity = new_capacity; m_size = m_capacity - pad_right; @@ -115,12 +115,10 @@ private: return; } - size_t padded_capacity = m_capacity + pad_right; - ProfileEvents::increment(ProfileEvents::IOBufferAllocs); - ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, padded_capacity); + ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, m_capacity); - size_t new_capacity = align(padded_capacity, alignment); + size_t new_capacity = align(m_capacity, alignment) + pad_right; m_data = static_cast(Allocator::alloc(new_capacity, alignment)); m_capacity = new_capacity; m_size = m_capacity - pad_right; From fb54d6b6e0d38a3ef9fd7635c6ee6a84072728be Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:07:49 +0300 Subject: [PATCH 786/950] BufferWithOwnMemory: take reallocs into account --- src/IO/BufferWithOwnMemory.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/IO/BufferWithOwnMemory.h b/src/IO/BufferWithOwnMemory.h index 17bd0df9993..0d571d6ae7c 100644 --- a/src/IO/BufferWithOwnMemory.h +++ b/src/IO/BufferWithOwnMemory.h @@ -89,6 +89,10 @@ struct Memory : boost::noncopyable, Allocator else { size_t new_capacity = align(new_size, alignment) + pad_right; + + size_t diff = new_capacity - m_capacity; + ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, diff); + m_data = static_cast(Allocator::realloc(m_data, m_capacity, new_capacity, alignment)); m_capacity = new_capacity; m_size = m_capacity - pad_right; From 14baed927f65e6e11426683a7d5f03d89bc11b46 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:07:49 +0300 Subject: [PATCH 787/950] Add extensive test for various read settings The following settings had been covered: - min_bytes_to_use_direct_io - local_filesystem_read_method - local_filesystem_read_prefetch - read_priority - max_read_buffer_size --- .../02051_read_settings.reference.j2 | 11 +++++++ .../0_stateless/02051_read_settings.sql.j2 | 31 +++++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 tests/queries/0_stateless/02051_read_settings.reference.j2 create mode 100644 tests/queries/0_stateless/02051_read_settings.sql.j2 diff --git a/tests/queries/0_stateless/02051_read_settings.reference.j2 b/tests/queries/0_stateless/02051_read_settings.reference.j2 new file mode 100644 index 00000000000..86aa67a9d2d --- /dev/null +++ b/tests/queries/0_stateless/02051_read_settings.reference.j2 @@ -0,0 +1,11 @@ +{% for read_method in ['read', 'mmap', 'pread_threadpool', 'pread_fake_async'] -%} +{% for direct_io in [0, 1] -%} +{% for prefetch in [0, 1] -%} +{% for priority in [0, 1] -%} +{% for buffer_size in [65505, 1048576] -%} +1000000 +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} +{% endfor -%} diff --git a/tests/queries/0_stateless/02051_read_settings.sql.j2 b/tests/queries/0_stateless/02051_read_settings.sql.j2 new file mode 100644 index 00000000000..9f02274e732 --- /dev/null +++ b/tests/queries/0_stateless/02051_read_settings.sql.j2 @@ -0,0 +1,31 @@ +-- Tags: long +-- +-- Test for testing various read settings. + +drop table if exists data_02051; + +create table data_02051 (key Int, value String) engine=MergeTree() order by key +as select number, repeat(toString(number), 5) from numbers(1e6); + +{# check each local_filesystem_read_method #} +{% for read_method in ['read', 'mmap', 'pread_threadpool', 'pread_fake_async'] %} +{# check w/ O_DIRECT and w/o (min_bytes_to_use_direct_io) #} +{% for direct_io in [0, 1] %} +{# check local_filesystem_read_prefetch (just a smoke test) #} +{% for prefetch in [0, 1] %} +{# check read_priority (just a smoke test) #} +{% for priority in [0, 1] %} +{# check alignment for O_DIRECT with various max_read_buffer_size #} +{% for buffer_size in [65505, 1048576] %} +select count(ignore(*)) from data_02051 settings + min_bytes_to_use_direct_io={{ direct_io }}, + local_filesystem_read_method='{{ read_method }}', + local_filesystem_read_prefetch={{ prefetch }}, + read_priority={{ priority }}, + max_read_buffer_size={{ buffer_size }} +; +{% endfor %} +{% endfor %} +{% endfor %} +{% endfor %} +{% endfor %} From 0e34a9d550cfe6924fe575871f36c44dd44acdaa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Oct 2021 23:47:51 +0300 Subject: [PATCH 788/950] Add MSan instrumentation for preadv2 MSan report [1]: ==10672==WARNING: MemorySanitizer: use-of-uninitialized-value 3 0x30dd6005 in DB::CompressionCodecFactory::get(unsigned char) const obj-x86_64-linux-gnu/../src/Compression/CompressionFactory.cpp:105:65 4 0x30dab4d5 in DB::CompressedReadBufferBase::readCompressedData(unsigned long&, unsigned long&, bool) obj-x86_64-linux-gnu/../src/Compression/CompressedReadBufferBase.cpp:127:53 5 0x30db54af in DB::CompressedReadBufferFromFile::readBig(char*, unsigned long) obj-x86_64-linux-gnu/../src/Compression/CompressedReadBufferFromFile.cpp:119:38 Uninitialized value was stored to memory at 0 0x9ef4e99 in __msan_memcpy (/src/ch/tmp/30191/clickhouse-msan+0x9ef4e99) 1 0xa0c1999 in DB::ReadBuffer::read() obj-x86_64-linux-gnu/../src/IO/ReadBuffer.h:173:13 2 0xa0c1999 in DB::ReadBuffer::readStrict() obj-x86_64-linux-gnu/../src/IO/ReadBuffer.h:184:27 3 0x30daafd1 in DB::CompressedReadBufferBase::readCompressedData() obj-x86_64-linux-gnu/../src/Compression/CompressedReadBufferBase.cpp:120:20 4 0x30db54af in DB::CompressedReadBufferFromFile::readBig() obj-x86_64-linux-gnu/../src/Compression/CompressedReadBufferFromFile.cpp:119:38 Uninitialized value was created by a heap allocation 0 0x9ef5285 in posix_memalign (/src/ch/tmp/30191/clickhouse-msan+0x9ef5285) 1 0xa0cb98d in Allocator::allocNoTrack(unsigned long, unsigned long) obj-x86_64-linux-gnu/../src/Common/Allocator.h:235:27 2 0xa0caf0d in Allocator::alloc(unsigned long, unsigned long) obj-x86_64-linux-gnu/../src/Common/Allocator.h:96:16 3 0xa0caf0d in DB::Memory >::alloc() obj-x86_64-linux-gnu/../src/IO/BufferWithOwnMemory.h:126:49 4 0xa0ca4a1 in DB::Memory >::Memory() obj-x86_64-linux-gnu/../src/IO/BufferWithOwnMemory.h:43:9 5 0xa0ca4a1 in DB::BufferWithOwnMemory::BufferWithOwnMemory(unsigned long, char*, unsigned long) obj-x86_64-linux-gnu/../src/IO/BufferWithOwnMemory.h:153:29 6 0xa0ca4a1 in DB::ReadBufferFromFileBase::ReadBufferFromFileBase() obj-x86_64-linux-gnu/../src/IO/ReadBufferFromFileBase.cpp:11:7 7 0xa59d980 in DB::AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescriptor() obj-x86_64-linux-gnu/../src/IO/AsynchronousReadBufferFromFileDescriptor.h:40:11 SUMMARY: MemorySanitizer: use-of-uninitialized-value obj-x86_64-linux-gnu/../contrib/libcxx/include/__hash_table:116:10 in std::__1::__constrain_hash(unsigned long, unsigned long) [1]: https://clickhouse-test-reports.s3.yandex.net/30191/18bb0e7d8d47dd694390f3f7b2ecc921a167afac/fuzzer_msan/report.html#fail1 --- src/IO/ThreadPoolReader.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ThreadPoolReader.cpp b/src/IO/ThreadPoolReader.cpp index 514075569f6..701fa759848 100644 --- a/src/IO/ThreadPoolReader.cpp +++ b/src/IO/ThreadPoolReader.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -151,6 +152,7 @@ std::future ThreadPoolReader::submit(Request reques else { bytes_read += res; + __msan_unpoison(request.buf, res); } } From f032fb71f0903e181e92479803a88fb482adbec1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 18 Oct 2021 02:09:22 +0300 Subject: [PATCH 789/950] Minor modification in hardware benchmark --- benchmark/hardware.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 76328e1509d..69e05cf804b 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -13,6 +13,7 @@ TRIES=3 AMD64_BIN_URL="https://builds.clickhouse.com/master/amd64/clickhouse" AARCH64_BIN_URL="https://builds.clickhouse.com/master/aarch64/clickhouse" +POWERPC64_BIN_URL="https://builds.clickhouse.com/master/ppc64le/clickhouse" # Note: on older Ubuntu versions, 'axel' does not support IPv6. If you are using IPv6-only servers on very old Ubuntu, just don't install 'axel'. @@ -38,6 +39,8 @@ if [[ ! -f clickhouse ]]; then $FASTER_DOWNLOAD "$AMD64_BIN_URL" elif [[ $CPU == aarch64 ]]; then $FASTER_DOWNLOAD "$AARCH64_BIN_URL" + elif [[ $CPU == powerpc64le ]]; then + $FASTER_DOWNLOAD "$POWERPC64_BIN_URL" else echo "Unsupported CPU type: $CPU" exit 1 @@ -52,7 +55,7 @@ fi if [[ ! -d data ]]; then if [[ ! -f $DATASET ]]; then - $FASTER_DOWNLOAD "https://clickhouse-datasets.s3.yandex.net/hits/partitions/$DATASET" + $FASTER_DOWNLOAD "https://datasets.clickhouse.com/hits/partitions/$DATASET" fi tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET From 2cef9983fa9f9d630055028a2350bb778b7618f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 18 Oct 2021 02:29:04 +0300 Subject: [PATCH 790/950] Remove recursive submodules --- cmake/find/amqpcpp.cmake | 2 +- cmake/find/avro.cmake | 2 +- cmake/find/base64.cmake | 4 ++-- cmake/find/brotli.cmake | 2 +- cmake/find/bzip2.cmake | 2 +- cmake/find/capnp.cmake | 2 +- cmake/find/cassandra.cmake | 2 +- cmake/find/cxx.cmake | 2 +- cmake/find/cyrus-sasl.cmake | 2 +- cmake/find/datasketches.cmake | 2 +- cmake/find/fast_float.cmake | 2 +- cmake/find/fastops.cmake | 2 +- cmake/find/grpc.cmake | 2 +- cmake/find/gtest.cmake | 2 +- cmake/find/h3.cmake | 2 +- cmake/find/hdfs3.cmake | 2 +- cmake/find/icu.cmake | 2 +- cmake/find/krb5.cmake | 2 +- cmake/find/ldap.cmake | 2 +- cmake/find/libgsasl.cmake | 2 +- cmake/find/libpqxx.cmake | 4 ++-- cmake/find/libprotobuf-mutator.cmake | 2 +- cmake/find/libuv.cmake | 2 +- cmake/find/libxml2.cmake | 2 +- cmake/find/llvm.cmake | 2 +- cmake/find/msgpack.cmake | 2 +- cmake/find/mysqlclient.cmake | 2 +- cmake/find/nanodbc.cmake | 2 +- cmake/find/nlp.cmake | 6 +++--- cmake/find/nuraft.cmake | 2 +- cmake/find/orc.cmake | 2 +- cmake/find/parquet.cmake | 2 +- cmake/find/protobuf.cmake | 2 +- cmake/find/rapidjson.cmake | 2 +- cmake/find/rdkafka.cmake | 4 ++-- cmake/find/re2.cmake | 2 +- cmake/find/rocksdb.cmake | 2 +- cmake/find/s2geometry.cmake | 2 +- cmake/find/s3.cmake | 2 +- cmake/find/sentry.cmake | 2 +- cmake/find/simdjson.cmake | 2 +- cmake/find/sqlite.cmake | 2 +- cmake/find/ssl.cmake | 2 +- cmake/find/stats.cmake | 4 ++-- cmake/find/xz.cmake | 2 +- cmake/find/yaml-cpp.cmake | 2 +- cmake/find/zlib.cmake | 2 +- cmake/find/zstd.cmake | 2 +- docker/test/fasttest/run.sh | 2 +- docs/en/development/developer-instruction.md | 12 ++++++------ docs/ja/development/developer-instruction.md | 12 ++++++------ docs/ru/development/developer-instruction.md | 12 ++++++------ docs/zh/development/developer-instruction.md | 12 ++++++------ utils/build/build_msvc2017.bat | 14 -------------- 54 files changed, 79 insertions(+), 93 deletions(-) delete mode 100644 utils/build/build_msvc2017.bat diff --git a/cmake/find/amqpcpp.cmake b/cmake/find/amqpcpp.cmake index 05e5d2da751..374e6dd6d7e 100644 --- a/cmake/find/amqpcpp.cmake +++ b/cmake/find/amqpcpp.cmake @@ -10,7 +10,7 @@ if (NOT ENABLE_AMQPCPP) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/CMakeLists.txt") - message (WARNING "submodule contrib/AMQP-CPP is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/AMQP-CPP is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal AMQP-CPP library") set (USE_AMQPCPP 0) return() diff --git a/cmake/find/avro.cmake b/cmake/find/avro.cmake index 74ccda3489f..351fa15d2d3 100644 --- a/cmake/find/avro.cmake +++ b/cmake/find/avro.cmake @@ -13,7 +13,7 @@ option (USE_INTERNAL_AVRO_LIBRARY if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/avro/lang/c++/CMakeLists.txt") if (USE_INTERNAL_AVRO_LIBRARY) - message(WARNING "submodule contrib/avro is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/avro is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find internal avro") set(USE_INTERNAL_AVRO_LIBRARY 0) endif() diff --git a/cmake/find/base64.cmake b/cmake/find/base64.cmake index acade11eb2f..ee12fbb11ba 100644 --- a/cmake/find/base64.cmake +++ b/cmake/find/base64.cmake @@ -10,11 +10,11 @@ endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64/LICENSE") set (MISSING_INTERNAL_BASE64_LIBRARY 1) - message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init") endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") - message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init") else() set (BASE64_LIBRARY base64) set (USE_BASE64 1) diff --git a/cmake/find/brotli.cmake b/cmake/find/brotli.cmake index bf498802922..4b2ee3d6de0 100644 --- a/cmake/find/brotli.cmake +++ b/cmake/find/brotli.cmake @@ -16,7 +16,7 @@ endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/brotli/c/include/brotli/decode.h") if (USE_INTERNAL_BROTLI_LIBRARY) - message (WARNING "submodule contrib/brotli is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/brotli is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot find internal brotli") set (USE_INTERNAL_BROTLI_LIBRARY 0) endif () diff --git a/cmake/find/bzip2.cmake b/cmake/find/bzip2.cmake index 15532a67c00..5e6a6fb5841 100644 --- a/cmake/find/bzip2.cmake +++ b/cmake/find/bzip2.cmake @@ -6,7 +6,7 @@ if (NOT ENABLE_BZIP2) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/bzip2/bzlib.h") - message (WARNING "submodule contrib/bzip2 is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/bzip2 is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal bzip2 library") set (USE_NLP 0) return() diff --git a/cmake/find/capnp.cmake b/cmake/find/capnp.cmake index ee4735bd175..b0e4cc419f6 100644 --- a/cmake/find/capnp.cmake +++ b/cmake/find/capnp.cmake @@ -11,7 +11,7 @@ option (USE_INTERNAL_CAPNP_LIBRARY "Set to FALSE to use system capnproto library if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/capnproto/CMakeLists.txt") if(USE_INTERNAL_CAPNP_LIBRARY) - message(WARNING "submodule contrib/capnproto is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/capnproto is missing. to fix try run: \n git submodule update --init") message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal capnproto") set(USE_INTERNAL_CAPNP_LIBRARY 0) endif() diff --git a/cmake/find/cassandra.cmake b/cmake/find/cassandra.cmake index b6e97ff5ef8..7fcbdbb90a5 100644 --- a/cmake/find/cassandra.cmake +++ b/cmake/find/cassandra.cmake @@ -14,7 +14,7 @@ if (APPLE) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cassandra") - message (ERROR "submodule contrib/cassandra is missing. to fix try run: \n git submodule update --init --recursive") + message (ERROR "submodule contrib/cassandra is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal Cassandra") set (USE_CASSANDRA 0) return() diff --git a/cmake/find/cxx.cmake b/cmake/find/cxx.cmake index b1da125e219..f38ac77b1ea 100644 --- a/cmake/find/cxx.cmake +++ b/cmake/find/cxx.cmake @@ -17,7 +17,7 @@ option (USE_INTERNAL_LIBCXX_LIBRARY "Disable to use system libcxx and libcxxabi if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcxx/CMakeLists.txt") if (USE_INTERNAL_LIBCXX_LIBRARY) - message(WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/libcxx is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libcxx") set(USE_INTERNAL_LIBCXX_LIBRARY 0) endif() diff --git a/cmake/find/cyrus-sasl.cmake b/cmake/find/cyrus-sasl.cmake index 974b8148fdc..f0c088995b0 100644 --- a/cmake/find/cyrus-sasl.cmake +++ b/cmake/find/cyrus-sasl.cmake @@ -6,7 +6,7 @@ endif() OPTION(ENABLE_CYRUS_SASL "Enable cyrus-sasl" ${DEFAULT_ENABLE_CYRUS_SASL}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cyrus-sasl/README") - message (WARNING "submodule contrib/cyrus-sasl is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/cyrus-sasl is missing. to fix try run: \n git submodule update --init") set (ENABLE_CYRUS_SASL 0) endif () diff --git a/cmake/find/datasketches.cmake b/cmake/find/datasketches.cmake index 44ef324a9f2..2d7e644890a 100644 --- a/cmake/find/datasketches.cmake +++ b/cmake/find/datasketches.cmake @@ -6,7 +6,7 @@ option (USE_INTERNAL_DATASKETCHES_LIBRARY "Set to FALSE to use system DataSketch if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/datasketches-cpp/theta/CMakeLists.txt") if (USE_INTERNAL_DATASKETCHES_LIBRARY) - message(WARNING "submodule contrib/datasketches-cpp is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/datasketches-cpp is missing. to fix try run: \n git submodule update --init") endif() set(MISSING_INTERNAL_DATASKETCHES_LIBRARY 1) set(USE_INTERNAL_DATASKETCHES_LIBRARY 0) diff --git a/cmake/find/fast_float.cmake b/cmake/find/fast_float.cmake index 4b215c710ad..3e8b7cc5280 100644 --- a/cmake/find/fast_float.cmake +++ b/cmake/find/fast_float.cmake @@ -1,5 +1,5 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fast_float/include/fast_float/fast_float.h") - message (FATAL_ERROR "submodule contrib/fast_float is missing. to fix try run: \n git submodule update --init --recursive") + message (FATAL_ERROR "submodule contrib/fast_float is missing. to fix try run: \n git submodule update --init") endif () set(FAST_FLOAT_LIBRARY fast_float) diff --git a/cmake/find/fastops.cmake b/cmake/find/fastops.cmake index 1675646654e..72426eb5912 100644 --- a/cmake/find/fastops.cmake +++ b/cmake/find/fastops.cmake @@ -10,7 +10,7 @@ if(NOT ENABLE_FASTOPS) endif() if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/fastops/fastops/fastops.h") - message(WARNING "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/fastops is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal fastops library") set(MISSING_INTERNAL_FASTOPS_LIBRARY 1) endif() diff --git a/cmake/find/grpc.cmake b/cmake/find/grpc.cmake index 1e440b3b350..f4b280876ef 100644 --- a/cmake/find/grpc.cmake +++ b/cmake/find/grpc.cmake @@ -26,7 +26,7 @@ option(USE_INTERNAL_GRPC_LIBRARY "Set to FALSE to use system gRPC library instea if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/CMakeLists.txt") if(USE_INTERNAL_GRPC_LIBRARY) - message(WARNING "submodule contrib/grpc is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/grpc is missing. to fix try run: \n git submodule update --init") message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal grpc") set(USE_INTERNAL_GRPC_LIBRARY 0) endif() diff --git a/cmake/find/gtest.cmake b/cmake/find/gtest.cmake index 9d4ab2608cb..c5f987d7368 100644 --- a/cmake/find/gtest.cmake +++ b/cmake/find/gtest.cmake @@ -4,7 +4,7 @@ option (USE_INTERNAL_GTEST_LIBRARY "Set to FALSE to use system Google Test inste if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest/CMakeLists.txt") if (USE_INTERNAL_GTEST_LIBRARY) - message (WARNING "submodule contrib/googletest is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/googletest is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal gtest") set (USE_INTERNAL_GTEST_LIBRARY 0) endif () diff --git a/cmake/find/h3.cmake b/cmake/find/h3.cmake index 03b6f32fc3c..e692b431e90 100644 --- a/cmake/find/h3.cmake +++ b/cmake/find/h3.cmake @@ -11,7 +11,7 @@ option(USE_INTERNAL_H3_LIBRARY "Set to FALSE to use system h3 library instead of if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib/include/h3Index.h") if(USE_INTERNAL_H3_LIBRARY) - message(WARNING "submodule contrib/h3 is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/h3 is missing. to fix try run: \n git submodule update --init") message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal h3 library") set(USE_INTERNAL_H3_LIBRARY 0) endif() diff --git a/cmake/find/hdfs3.cmake b/cmake/find/hdfs3.cmake index 3aab2b612ef..aac6b99dfa2 100644 --- a/cmake/find/hdfs3.cmake +++ b/cmake/find/hdfs3.cmake @@ -16,7 +16,7 @@ option(USE_INTERNAL_HDFS3_LIBRARY "Set to FALSE to use system HDFS3 instead of b if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include/hdfs/hdfs.h") if(USE_INTERNAL_HDFS3_LIBRARY) - message(WARNING "submodule contrib/libhdfs3 is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/libhdfs3 is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal HDFS3 library") set(USE_INTERNAL_HDFS3_LIBRARY 0) endif() diff --git a/cmake/find/icu.cmake b/cmake/find/icu.cmake index 40fb391656d..0b775a68eda 100644 --- a/cmake/find/icu.cmake +++ b/cmake/find/icu.cmake @@ -16,7 +16,7 @@ option (USE_INTERNAL_ICU_LIBRARY "Set to FALSE to use system ICU library instead if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/icu/icu4c/LICENSE") if (USE_INTERNAL_ICU_LIBRARY) - message (WARNING "submodule contrib/icu is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/icu is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ICU") set (USE_INTERNAL_ICU_LIBRARY 0) endif () diff --git a/cmake/find/krb5.cmake b/cmake/find/krb5.cmake index 49b7462b710..24cc51325dc 100644 --- a/cmake/find/krb5.cmake +++ b/cmake/find/krb5.cmake @@ -1,7 +1,7 @@ OPTION(ENABLE_KRB5 "Enable krb5" ${ENABLE_LIBRARIES}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/krb5/README") - message (WARNING "submodule contrib/krb5 is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/krb5 is missing. to fix try run: \n git submodule update --init") set (ENABLE_KRB5 0) endif () diff --git a/cmake/find/ldap.cmake b/cmake/find/ldap.cmake index 71222d26c66..0d14e2c4199 100644 --- a/cmake/find/ldap.cmake +++ b/cmake/find/ldap.cmake @@ -15,7 +15,7 @@ option (USE_INTERNAL_LDAP_LIBRARY "Set to FALSE to use system *LDAP library inst if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/openldap/README") if (USE_INTERNAL_LDAP_LIBRARY) - message (WARNING "Submodule contrib/openldap is missing. To fix try running:\n git submodule update --init --recursive") + message (WARNING "Submodule contrib/openldap is missing. To fix try running:\n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal LDAP library") endif () diff --git a/cmake/find/libgsasl.cmake b/cmake/find/libgsasl.cmake index 3c742af2566..3aec5c0c30a 100644 --- a/cmake/find/libgsasl.cmake +++ b/cmake/find/libgsasl.cmake @@ -16,7 +16,7 @@ endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h") if (USE_INTERNAL_LIBGSASL_LIBRARY) - message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libgsasl") set (USE_INTERNAL_LIBGSASL_LIBRARY 0) endif () diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake index f981df19aaa..b2a1e217b10 100644 --- a/cmake/find/libpqxx.cmake +++ b/cmake/find/libpqxx.cmake @@ -5,14 +5,14 @@ if (NOT ENABLE_LIBPQXX) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/CMakeLists.txt") - message (WARNING "submodule contrib/libpqxx is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/libpqxx is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpqxx library") set (USE_LIBPQXX 0) return() endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpq/include") - message (ERROR "submodule contrib/libpq is missing. to fix try run: \n git submodule update --init --recursive") + message (ERROR "submodule contrib/libpq is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpq needed for libpqxx") set (USE_LIBPQXX 0) return() diff --git a/cmake/find/libprotobuf-mutator.cmake b/cmake/find/libprotobuf-mutator.cmake index 8aa595230cd..a308db67c8b 100644 --- a/cmake/find/libprotobuf-mutator.cmake +++ b/cmake/find/libprotobuf-mutator.cmake @@ -7,5 +7,5 @@ endif() set(LibProtobufMutator_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libprotobuf-mutator") if (NOT EXISTS "${LibProtobufMutator_SOURCE_DIR}/README.md") - message (ERROR "submodule contrib/libprotobuf-mutator is missing. to fix try run: \n git submodule update --init --recursive") + message (ERROR "submodule contrib/libprotobuf-mutator is missing. to fix try run: \n git submodule update --init") endif() diff --git a/cmake/find/libuv.cmake b/cmake/find/libuv.cmake index f0023209309..c94dfd50b76 100644 --- a/cmake/find/libuv.cmake +++ b/cmake/find/libuv.cmake @@ -5,7 +5,7 @@ if (OS_DARWIN AND COMPILER_GCC) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libuv") - message (WARNING "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/libuv is missing. to fix try run: \n git submodule update --init") SET(MISSING_INTERNAL_LIBUV_LIBRARY 1) return() endif() diff --git a/cmake/find/libxml2.cmake b/cmake/find/libxml2.cmake index cdf079c33d2..8f7e79d84c9 100644 --- a/cmake/find/libxml2.cmake +++ b/cmake/find/libxml2.cmake @@ -2,7 +2,7 @@ option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h") if (USE_INTERNAL_LIBXML2_LIBRARY) - message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libxml") set (USE_INTERNAL_LIBXML2_LIBRARY 0) endif () diff --git a/cmake/find/llvm.cmake b/cmake/find/llvm.cmake index 84ac29991ab..ece5d5434a0 100644 --- a/cmake/find/llvm.cmake +++ b/cmake/find/llvm.cmake @@ -12,7 +12,7 @@ if (NOT ENABLE_EMBEDDED_COMPILER) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/llvm/llvm/CMakeLists.txt") - message (${RECONFIGURE_MESSAGE_LEVEL} "submodule /contrib/llvm is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "submodule /contrib/llvm is missing. to fix try run: \n git submodule update --init") endif () set (USE_EMBEDDED_COMPILER 1) diff --git a/cmake/find/msgpack.cmake b/cmake/find/msgpack.cmake index 130aa007ad5..c15fedd0e30 100644 --- a/cmake/find/msgpack.cmake +++ b/cmake/find/msgpack.cmake @@ -11,7 +11,7 @@ option (USE_INTERNAL_MSGPACK_LIBRARY "Set to FALSE to use system msgpack library if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/msgpack-c/include/msgpack.hpp") if(USE_INTERNAL_MSGPACK_LIBRARY) - message(WARNING "Submodule contrib/msgpack-c is missing. To fix try run: \n git submodule update --init --recursive") + message(WARNING "Submodule contrib/msgpack-c is missing. To fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal msgpack") set(USE_INTERNAL_MSGPACK_LIBRARY 0) endif() diff --git a/cmake/find/mysqlclient.cmake b/cmake/find/mysqlclient.cmake index 634681d98f6..0af03676d71 100644 --- a/cmake/find/mysqlclient.cmake +++ b/cmake/find/mysqlclient.cmake @@ -16,7 +16,7 @@ option(USE_INTERNAL_MYSQL_LIBRARY "Set to FALSE to use system mysqlclient librar if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/mariadb-connector-c/README") if(USE_INTERNAL_MYSQL_LIBRARY) - message(WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/mariadb-connector-c is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal mysql library") set(USE_INTERNAL_MYSQL_LIBRARY 0) endif() diff --git a/cmake/find/nanodbc.cmake b/cmake/find/nanodbc.cmake index 894a2a60bad..d48e294c9e5 100644 --- a/cmake/find/nanodbc.cmake +++ b/cmake/find/nanodbc.cmake @@ -7,7 +7,7 @@ if (NOT USE_INTERNAL_NANODBC_LIBRARY) endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/nanodbc/CMakeLists.txt") - message (FATAL_ERROR "submodule contrib/nanodbc is missing. to fix try run: \n git submodule update --init --recursive") + message (FATAL_ERROR "submodule contrib/nanodbc is missing. to fix try run: \n git submodule update --init") endif() set (NANODBC_LIBRARY nanodbc) diff --git a/cmake/find/nlp.cmake b/cmake/find/nlp.cmake index f1204a85dea..efa9b39ddae 100644 --- a/cmake/find/nlp.cmake +++ b/cmake/find/nlp.cmake @@ -7,21 +7,21 @@ if (NOT ENABLE_NLP) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libstemmer_c/Makefile") - message (WARNING "submodule contrib/libstemmer_c is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/libstemmer_c is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libstemmer_c library, NLP functions will be disabled") set (USE_NLP 0) return() endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/wordnet-blast/CMakeLists.txt") - message (WARNING "submodule contrib/wordnet-blast is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/wordnet-blast is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal wordnet-blast library, NLP functions will be disabled") set (USE_NLP 0) return() endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/lemmagen-c/README.md") - message (WARNING "submodule contrib/lemmagen-c is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/lemmagen-c is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal lemmagen-c library, NLP functions will be disabled") set (USE_NLP 0) return() diff --git a/cmake/find/nuraft.cmake b/cmake/find/nuraft.cmake index 4e5258e132f..59caa9e7373 100644 --- a/cmake/find/nuraft.cmake +++ b/cmake/find/nuraft.cmake @@ -5,7 +5,7 @@ if (NOT ENABLE_NURAFT) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/NuRaft/CMakeLists.txt") - message (WARNING "submodule contrib/NuRaft is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/NuRaft is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal NuRaft library") set (USE_NURAFT 0) return() diff --git a/cmake/find/orc.cmake b/cmake/find/orc.cmake index 01734224a6a..a5c3f57468a 100644 --- a/cmake/find/orc.cmake +++ b/cmake/find/orc.cmake @@ -18,7 +18,7 @@ include(cmake/find/snappy.cmake) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include/orc/OrcFile.hh") if(USE_INTERNAL_ORC_LIBRARY) - message(WARNING "submodule contrib/orc is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/orc is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ORC") set(USE_INTERNAL_ORC_LIBRARY 0) endif() diff --git a/cmake/find/parquet.cmake b/cmake/find/parquet.cmake index eb1b529fbfe..4b56a829df5 100644 --- a/cmake/find/parquet.cmake +++ b/cmake/find/parquet.cmake @@ -20,7 +20,7 @@ endif() if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/CMakeLists.txt") if(USE_INTERNAL_PARQUET_LIBRARY) - message(WARNING "submodule contrib/arrow (required for Parquet) is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/arrow (required for Parquet) is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal parquet library") set(USE_INTERNAL_PARQUET_LIBRARY 0) endif() diff --git a/cmake/find/protobuf.cmake b/cmake/find/protobuf.cmake index eb9fbe3edef..096288fd2ab 100644 --- a/cmake/find/protobuf.cmake +++ b/cmake/find/protobuf.cmake @@ -15,7 +15,7 @@ option(USE_INTERNAL_PROTOBUF_LIBRARY "Set to FALSE to use system protobuf instea if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/protobuf/cmake/CMakeLists.txt") if(USE_INTERNAL_PROTOBUF_LIBRARY) - message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init") message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf") set(USE_INTERNAL_PROTOBUF_LIBRARY 0) endif() diff --git a/cmake/find/rapidjson.cmake b/cmake/find/rapidjson.cmake index f880d19143e..62db4695c58 100644 --- a/cmake/find/rapidjson.cmake +++ b/cmake/find/rapidjson.cmake @@ -10,7 +10,7 @@ option(USE_INTERNAL_RAPIDJSON_LIBRARY "Set to FALSE to use system rapidjson libr if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rapidjson/include/rapidjson/rapidjson.h") if(USE_INTERNAL_RAPIDJSON_LIBRARY) - message(WARNING "submodule contrib/rapidjson is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/rapidjson is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal rapidjson library") set(USE_INTERNAL_RAPIDJSON_LIBRARY 0) endif() diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index f6460c1d9a3..5b370a42cdc 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -11,7 +11,7 @@ option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka inst if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt") if(USE_INTERNAL_RDKAFKA_LIBRARY) - message (WARNING "submodule contrib/cppkafka is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/cppkafka is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal cppkafka") set (USE_INTERNAL_RDKAFKA_LIBRARY 0) endif() @@ -20,7 +20,7 @@ endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/CMakeLists.txt") if(USE_INTERNAL_RDKAFKA_LIBRARY OR MISSING_INTERNAL_CPPKAFKA_LIBRARY) - message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal rdkafka") set (USE_INTERNAL_RDKAFKA_LIBRARY 0) endif() diff --git a/cmake/find/re2.cmake b/cmake/find/re2.cmake index 87bc974c788..09240f33f7d 100644 --- a/cmake/find/re2.cmake +++ b/cmake/find/re2.cmake @@ -2,7 +2,7 @@ option (USE_INTERNAL_RE2_LIBRARY "Set to FALSE to use system re2 library instead if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/re2/CMakeLists.txt") if(USE_INTERNAL_RE2_LIBRARY) - message(WARNING "submodule contrib/re2 is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/re2 is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal re2 library") endif() set(USE_INTERNAL_RE2_LIBRARY 0) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 94278a603d7..109eabc271b 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -15,7 +15,7 @@ option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") if (USE_INTERNAL_ROCKSDB_LIBRARY) - message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init") message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal rocksdb") endif() set (MISSING_INTERNAL_ROCKSDB 1) diff --git a/cmake/find/s2geometry.cmake b/cmake/find/s2geometry.cmake index 2364c6ba193..348805b342e 100644 --- a/cmake/find/s2geometry.cmake +++ b/cmake/find/s2geometry.cmake @@ -3,7 +3,7 @@ option(ENABLE_S2_GEOMETRY "Enable S2 geometry library" ${ENABLE_LIBRARIES}) if (ENABLE_S2_GEOMETRY) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/s2geometry") - message (WARNING "submodule contrib/s2geometry is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/s2geometry is missing. to fix try run: \n git submodule update --init") set (ENABLE_S2_GEOMETRY 0) set (USE_S2_GEOMETRY 0) else() diff --git a/cmake/find/s3.cmake b/cmake/find/s3.cmake index a2ed3e416d0..9a10c3f13ef 100644 --- a/cmake/find/s3.cmake +++ b/cmake/find/s3.cmake @@ -23,7 +23,7 @@ if (NOT USE_INTERNAL_AWS_S3_LIBRARY) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/aws/aws-cpp-sdk-s3") - message (WARNING "submodule contrib/aws is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/aws is missing. to fix try run: \n git submodule update --init") if (USE_INTERNAL_AWS_S3_LIBRARY) message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal S3 library") endif () diff --git a/cmake/find/sentry.cmake b/cmake/find/sentry.cmake index a986599abce..4283e75f9ef 100644 --- a/cmake/find/sentry.cmake +++ b/cmake/find/sentry.cmake @@ -2,7 +2,7 @@ set (SENTRY_LIBRARY "sentry") set (SENTRY_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/sentry-native/include") if (NOT EXISTS "${SENTRY_INCLUDE_DIR}/sentry.h") - message (WARNING "submodule contrib/sentry-native is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/sentry-native is missing. to fix try run: \n git submodule update --init") if (USE_SENTRY) message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal sentry library") endif() diff --git a/cmake/find/simdjson.cmake b/cmake/find/simdjson.cmake index cffe20bdb2d..bf22a331f04 100644 --- a/cmake/find/simdjson.cmake +++ b/cmake/find/simdjson.cmake @@ -1,7 +1,7 @@ option (USE_SIMDJSON "Use simdjson" ${ENABLE_LIBRARIES}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson.h") - message (WARNING "submodule contrib/simdjson is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/simdjson is missing. to fix try run: \n git submodule update --init") if (USE_SIMDJSON) message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal simdjson library") endif() diff --git a/cmake/find/sqlite.cmake b/cmake/find/sqlite.cmake index cfa33fdebbb..083a9faea59 100644 --- a/cmake/find/sqlite.cmake +++ b/cmake/find/sqlite.cmake @@ -5,7 +5,7 @@ if (NOT ENABLE_SQLITE) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/sqlite-amalgamation/sqlite3.c") - message (WARNING "submodule contrib/sqlite3-amalgamation is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/sqlite3-amalgamation is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal sqlite library") set (USE_SQLITE 0) return() diff --git a/cmake/find/ssl.cmake b/cmake/find/ssl.cmake index fdc0bfb27d3..fb411b93593 100644 --- a/cmake/find/ssl.cmake +++ b/cmake/find/ssl.cmake @@ -13,7 +13,7 @@ option(USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/boringssl/README.md") if(USE_INTERNAL_SSL_LIBRARY) - message(WARNING "submodule contrib/boringssl is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/boringssl is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal ssl library") endif() set(USE_INTERNAL_SSL_LIBRARY 0) diff --git a/cmake/find/stats.cmake b/cmake/find/stats.cmake index dea108ed920..589da1603d5 100644 --- a/cmake/find/stats.cmake +++ b/cmake/find/stats.cmake @@ -2,11 +2,11 @@ option(ENABLE_STATS "Enable StatsLib library" ${ENABLE_LIBRARIES}) if (ENABLE_STATS) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/stats") - message (WARNING "submodule contrib/stats is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/stats is missing. to fix try run: \n git submodule update --init") set (ENABLE_STATS 0) set (USE_STATS 0) elseif (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/gcem") - message (WARNING "submodule contrib/gcem is missing. to fix try run: \n git submodule update --init --recursive") + message (WARNING "submodule contrib/gcem is missing. to fix try run: \n git submodule update --init") set (ENABLE_STATS 0) set (USE_STATS 0) else() diff --git a/cmake/find/xz.cmake b/cmake/find/xz.cmake index 0d19859c6b1..f4c230859bc 100644 --- a/cmake/find/xz.cmake +++ b/cmake/find/xz.cmake @@ -2,7 +2,7 @@ option (USE_INTERNAL_XZ_LIBRARY "Set to OFF to use system xz (lzma) library inst if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api/lzma.h") if(USE_INTERNAL_XZ_LIBRARY) - message(WARNING "submodule contrib/xz is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/xz is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal xz (lzma) library") set(USE_INTERNAL_XZ_LIBRARY 0) endif() diff --git a/cmake/find/yaml-cpp.cmake b/cmake/find/yaml-cpp.cmake index 4633d559980..2aba6808e31 100644 --- a/cmake/find/yaml-cpp.cmake +++ b/cmake/find/yaml-cpp.cmake @@ -5,5 +5,5 @@ if (NOT USE_YAML_CPP) endif() if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/yaml-cpp/README.md") - message (ERROR "submodule contrib/yaml-cpp is missing. to fix try run: \n git submodule update --init --recursive") + message (ERROR "submodule contrib/yaml-cpp is missing. to fix try run: \n git submodule update --init") endif() diff --git a/cmake/find/zlib.cmake b/cmake/find/zlib.cmake index f66f9e6713d..50a5bc63d15 100644 --- a/cmake/find/zlib.cmake +++ b/cmake/find/zlib.cmake @@ -12,7 +12,7 @@ endif () if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}/zlib.h") if(USE_INTERNAL_ZLIB_LIBRARY) - message(WARNING "submodule contrib/${INTERNAL_ZLIB_NAME} is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/${INTERNAL_ZLIB_NAME} is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal zlib library") endif() set(USE_INTERNAL_ZLIB_LIBRARY 0) diff --git a/cmake/find/zstd.cmake b/cmake/find/zstd.cmake index b12bb701e0e..812e1eed139 100644 --- a/cmake/find/zstd.cmake +++ b/cmake/find/zstd.cmake @@ -2,7 +2,7 @@ option (USE_INTERNAL_ZSTD_LIBRARY "Set to FALSE to use system zstd library inste if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/zstd/lib/zstd.h") if(USE_INTERNAL_ZSTD_LIBRARY) - message(WARNING "submodule contrib/zstd is missing. to fix try run: \n git submodule update --init --recursive") + message(WARNING "submodule contrib/zstd is missing. to fix try run: \n git submodule update --init") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal zstd library") set(USE_INTERNAL_ZSTD_LIBRARY 0) endif() diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index f4b99603554..2e37522f1b4 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -189,7 +189,7 @@ function clone_submodules ) git submodule sync - git submodule update --depth 1 --init --recursive "${SUBMODULES_TO_UPDATE[@]}" + git submodule update --depth 1 --init "${SUBMODULES_TO_UPDATE[@]}" git submodule foreach git reset --hard git submodule foreach git checkout @ -f git submodule foreach git clean -xfd diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index be6a08d397c..024ce27d60d 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -37,7 +37,7 @@ Next, you need to download the source files onto your working machine. This is c In the command line terminal run: - git clone --recursive git@github.com:your_github_username/ClickHouse.git + git clone git@github.com:your_github_username/ClickHouse.git cd ClickHouse Note: please, substitute *your_github_username* with what is appropriate! @@ -79,7 +79,7 @@ After successfully running this command you will be able to pull updates from th Working with submodules in git could be painful. Next commands will help to manage it: - # ! each command accepts --recursive + # ! each command accepts # Update remote URLs for submodules. Barely rare case git submodule sync # Add new submodules @@ -92,16 +92,16 @@ Working with submodules in git could be painful. Next commands will help to mana The next commands would help you to reset all submodules to the initial state (!WARNING! - any changes inside will be deleted): # Synchronizes submodules' remote URL with .gitmodules - git submodule sync --recursive + git submodule sync # Update the registered submodules with initialize not yet initialized - git submodule update --init --recursive + git submodule update --init # Reset all changes done after HEAD git submodule foreach git reset --hard # Clean files from .gitignore git submodule foreach git clean -xfd # Repeat last 4 commands for all submodule - git submodule foreach git submodule sync --recursive - git submodule foreach git submodule update --init --recursive + git submodule foreach git submodule sync + git submodule foreach git submodule update --init git submodule foreach git submodule foreach git reset --hard git submodule foreach git submodule foreach git clean -xfd diff --git a/docs/ja/development/developer-instruction.md b/docs/ja/development/developer-instruction.md index 3eafbf0481d..c95dc0e2ea4 100644 --- a/docs/ja/development/developer-instruction.md +++ b/docs/ja/development/developer-instruction.md @@ -41,7 +41,7 @@ Ubuntuでこれを行うには、コマンドラインターミナルで実行 コマンドラインターミナルで実行: - git clone --recursive git@github.com:your_github_username/ClickHouse.git + git clone git@github.com:your_github_username/ClickHouse.git cd ClickHouse 注:、代理して下さい *your_github_username* 適切なもので! @@ -83,7 +83,7 @@ ClickHouseリポジトリは以下を使用します `submodules`. That is what Gitでサブモジュールを操作するのは苦痛です。 次のコマンドは管理に役立ちます: - # ! each command accepts --recursive + # ! each command accepts # Update remote URLs for submodules. Barely rare case git submodule sync # Add new submodules @@ -96,16 +96,16 @@ Gitでサブモジュールを操作するのは苦痛です。 次のコマン 次のコマンドは、すべてのサブモジュールを初期状態にリセットするのに役立ちます(!ツづツつキツ。 -内部の変更は削除されます): # Synchronizes submodules' remote URL with .gitmodules - git submodule sync --recursive + git submodule sync # Update the registered submodules with initialize not yet initialized - git submodule update --init --recursive + git submodule update --init # Reset all changes done after HEAD git submodule foreach git reset --hard # Clean files from .gitignore git submodule foreach git clean -xfd # Repeat last 4 commands for all submodule - git submodule foreach git submodule sync --recursive - git submodule foreach git submodule update --init --recursive + git submodule foreach git submodule sync + git submodule foreach git submodule update --init git submodule foreach git submodule foreach git reset --hard git submodule foreach git submodule foreach git clean -xfd diff --git a/docs/ru/development/developer-instruction.md b/docs/ru/development/developer-instruction.md index 215a13a465e..8466c709ad1 100644 --- a/docs/ru/development/developer-instruction.md +++ b/docs/ru/development/developer-instruction.md @@ -40,7 +40,7 @@ ClickHouse не работает и не собирается на 32-битны Выполните в терминале: - git clone --recursive git@github.com:ClickHouse/ClickHouse.git + git clone git@github.com:ClickHouse/ClickHouse.git cd ClickHouse Замените первое вхождение слова `ClickHouse` в команде для git на имя вашего аккаунта на GitHub. @@ -82,7 +82,7 @@ ClickHouse не работает и не собирается на 32-битны Работа с сабмодулями git может быть достаточно болезненной. Следующие команды позволят содержать их в порядке: - # ! Каждая команда принимает аргумент --recursive + # ! Каждая команда принимает аргумент # Обновить URLs удалённого репозитория для каждого сабмодуля, используется относительно редко git submodule sync # Добавить новые сабмодули @@ -96,16 +96,16 @@ ClickHouse не работает и не собирается на 32-битны # Synchronizes submodules' remote URL with .gitmodules # Обновить URLs удалённого репозитория для каждого сабмодуля - git submodule sync --recursive + git submodule sync # Обновить существующие модули и добавить отсутствующие - git submodule update --init --recursive + git submodule update --init # Удалить все изменения в сабмодуле относительно HEAD git submodule foreach git reset --hard # Очистить игнорируемые файлы git submodule foreach git clean -xfd # Повторить последние 4 команды для каждого из сабмодулей - git submodule foreach git submodule sync --recursive - git submodule foreach git submodule update --init --recursive + git submodule foreach git submodule sync + git submodule foreach git submodule update --init git submodule foreach git submodule foreach git reset --hard git submodule foreach git submodule foreach git clean -xfd diff --git a/docs/zh/development/developer-instruction.md b/docs/zh/development/developer-instruction.md index e37533fb36b..bd7a197f926 100644 --- a/docs/zh/development/developer-instruction.md +++ b/docs/zh/development/developer-instruction.md @@ -29,7 +29,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 在终端命令行输入下列指令: - git clone --recursive git@guthub.com:your_github_username/ClickHouse.git + git clone git@guthub.com:your_github_username/ClickHouse.git cd ClickHouse 请注意,您需要将*your_github_username* 替换成实际使用的账户名! @@ -71,7 +71,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 在git中使用子模块可能会很痛苦。 接下来的命令将有助于管理它: - # ! each command accepts --recursive + # ! each command accepts # Update remote URLs for submodules. Barely rare case git submodule sync # Add new submodules @@ -84,16 +84,16 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 接下来的命令将帮助您将所有子模块重置为初始状态(!华林! -里面的任何chenges将被删除): # Synchronizes submodules' remote URL with .gitmodules - git submodule sync --recursive + git submodule sync # Update the registered submodules with initialize not yet initialized - git submodule update --init --recursive + git submodule update --init # Reset all changes done after HEAD git submodule foreach git reset --hard # Clean files from .gitignore git submodule foreach git clean -xfd # Repeat last 4 commands for all submodule - git submodule foreach git submodule sync --recursive - git submodule foreach git submodule update --init --recursive + git submodule foreach git submodule sync + git submodule foreach git submodule update --init git submodule foreach git submodule foreach git reset --hard git submodule foreach git submodule foreach git clean -xfd diff --git a/utils/build/build_msvc2017.bat b/utils/build/build_msvc2017.bat deleted file mode 100644 index 880802999ab..00000000000 --- a/utils/build/build_msvc2017.bat +++ /dev/null @@ -1,14 +0,0 @@ - -:: WINDOWS BUILD NOT SUPPORTED! -:: Script only for development - -cd ../.. -git clone --recursive https://github.com/madler/zlib contrib/zlib -md build -cd build - -:: Stage 1: try build client -cmake .. -G "Visual Studio 15 2017 Win64" -DENABLE_CLICKHOUSE_ALL=0 -DENABLE_CLICKHOUSE_CLIENT=1 > cmake.log -cmake --build . --target clickhouse -- /m > build.log -:: Stage 2: try build minimal server -:: Stage 3: enable all possible features (ssl, ...) From ab9d5d8cc789438ab0b01f6b0a4d712e190fed6f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Oct 2021 06:06:38 +0000 Subject: [PATCH 791/950] Better --- programs/local/LocalServer.cpp | 5 ----- src/Databases/DatabaseOnDisk.cpp | 9 +++++++++ src/Interpreters/loadMetadata.cpp | 2 +- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0c5f64ea913..cdd5ae13f99 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -579,11 +579,6 @@ void LocalServer::processConfig() { String path = global_context->getPath(); - /// When tables are loaded from .sql we initialize background executors - /// regardless there are MergeTree tables or not, because no better place was found. - /// In other cases it will be initialized only when there are mergeTree tables. - global_context->initializeBackgroundExecutors(); - /// Lock path directory before read status.emplace(fs::path(path) / "status", StatusFile::write_full_info); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 97e59f53f64..a71d539e3c5 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -53,6 +53,15 @@ std::pair createTableFromAST( ast_create_query.attach = true; ast_create_query.database = database_name; + auto global_context = context->getGlobalContext(); + if (global_context + && global_context->getApplicationType() == Context::ApplicationType::LOCAL + && !global_context->isBackgroundExecutorsInitialized() + && ast_create_query.storage && endsWith(ast_create_query.storage->engine->name, "MergeTree")) + { + global_context->initializeBackgroundExecutors(); + } + if (ast_create_query.as_table_function) { const auto & factory = TableFunctionFactory::instance(); diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 6a3db48e835..65b2065b2ad 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -161,7 +161,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam bool create_default_db_if_not_exists = !default_database_name.empty(); bool metadata_dir_for_default_db_already_exists = databases.count(default_database_name); if (create_default_db_if_not_exists && !metadata_dir_for_default_db_already_exists) - databases.emplace(default_database_name, path + "/" + escapeForFileName(default_database_name)); + databases.emplace(default_database_name, std::filesystem::path(path) / escapeForFileName(default_database_name)); TablesLoader::Databases loaded_databases; for (const auto & [name, db_path] : databases) From 074e02eb1433828138ca06a11c259c6a994841f0 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 06:38:47 +0000 Subject: [PATCH 792/950] fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 34 ++--- src/Storages/FileLog/DirectoryWatcherBase.h | 3 +- .../FileLog/FileLogDirectoryWatcher.cpp | 2 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- .../FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 47 +++--- src/Storages/FileLog/StorageFileLog.h | 33 ++-- .../02023_storage_filelog.reference | 143 ++++++++++++++++++ .../0_stateless/02023_storage_filelog.sh | 19 ++- .../0_stateless/02024_storage_filelog_mv.sh | 9 +- 10 files changed, 228 insertions(+), 66 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index cad2072a8e6..b5d914f700d 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include @@ -20,7 +22,11 @@ static constexpr int buffer_size = 4096; DirectoryWatcherBase::DirectoryWatcherBase( FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) - : WithContext(context_), owner(owner_), path(path_), event_mask(event_mask_) + : WithContext(context_) + , owner(owner_) + , path(path_) + , event_mask(event_mask_) + , milliseconds_to_wait(owner.storage.getFileLogSettings()->poll_directory_watch_events_backoff_init.totalMilliseconds()) { if (!std::filesystem::exists(path)) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Path {} does not exist", path); @@ -64,8 +70,9 @@ void DirectoryWatcherBase::watchFunc() while (!stopped) { const auto & settings = owner.storage.getFileLogSettings(); - if (poll(&pfd, 1, 500) > 0 && pfd.revents & POLLIN) + if (poll(&pfd, 1, milliseconds_to_wait) > 0 && pfd.revents & POLLIN) { + milliseconds_to_wait = settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); int n = read(fd, buffer.data(), buffer.size()); int i = 0; if (n > 0) @@ -109,33 +116,20 @@ void DirectoryWatcherBase::watchFunc() } /// Wake up reader thread - auto & mutex = owner.storage.getMutex(); - auto & cv = owner.storage.getConditionVariable(); - std::unique_lock lock(mutex); - owner.storage.setNewEvents(); - lock.unlock(); - cv.notify_one(); + owner.storage.wakeUp(); } - else - { - if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) - milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; - break; + else + { + if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; } } - - if (!stopped) - watch_task->scheduleAfter(milliseconds_to_wait); } - DirectoryWatcherBase::~DirectoryWatcherBase() { stop(); close(fd); - - if (watch_task) - watch_task->deactivate(); } void DirectoryWatcherBase::start() diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index 88f864ac17d..a640f686c8a 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -97,10 +97,11 @@ private: std::atomic stopped{false}; - uint64_t milliseconds_to_wait; const std::string path; int event_mask; + uint64_t milliseconds_to_wait; + int fd; }; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index c4a153dbfe7..192721f9f3c 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -78,9 +78,9 @@ void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEven auto event_path = ev.path; EventInfo info{ev.event, "onItemModified"}; - /// Already have MODIFY event for this file if (auto it = events.find(event_path); it != events.end()) { + /// Already have MODIFY event for this file if (it->second.received_modification_event) return; else diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 8e0ecf5358c..0b0c86397aa 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include @@ -10,6 +9,7 @@ namespace DB { +class StorageFileLog; class FileLogDirectoryWatcher { diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 384de64cd3f..a55df9fe09e 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -117,7 +117,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ { /// Need to get offset before reading record from stream auto offset = reader.tellg(); - if (static_cast(offset) < file_meta.last_open_end) + if (static_cast(offset) >= file_meta.last_open_end) break; record.offset = offset; StorageFileLog::assertStreamGood(reader); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 66add8128ef..81303f623aa 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -47,8 +47,6 @@ namespace ErrorCodes namespace { - const auto RESCHEDULE_MS = 500; - const auto BACKOFF_TRESHOLD = 32000; const auto MAX_THREAD_WORK_DURATION_MS = 60000; } @@ -57,7 +55,6 @@ StorageFileLog::StorageFileLog( ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, const String & comment, @@ -66,10 +63,9 @@ StorageFileLog::StorageFileLog( , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) , path(path_) - , relative_data_path(relative_data_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) - , milliseconds_to_wait(RESCHEDULE_MS) + , milliseconds_to_wait(filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds()) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -100,9 +96,9 @@ StorageFileLog::StorageFileLog( void StorageFileLog::loadMetaFiles(bool attach) { - const auto & storage_id = getStorageID(); - root_meta_path = std::filesystem::path(getContext()->getPath()) / "metadata" / "filelog_storage_metadata" / storage_id.getDatabaseName() - / storage_id.getTableName(); + const auto & storage = getStorageID(); + root_meta_path + = std::filesystem::path(getContext()->getPath()) / ".filelog_storage_metadata" / storage.getDatabaseName() / storage.getTableName(); /// Attach table if (attach) @@ -110,8 +106,8 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Meta file may lost, log and create directory if (!std::filesystem::exists(root_meta_path)) { + /// Create root_meta_path directory when store meta data LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName()); - std::filesystem::create_directories(root_meta_path); } /// Load all meta info to file_infos; deserialize(); @@ -180,8 +176,8 @@ void StorageFileLog::loadFiles() /// data file have been renamed, need update meta file's name if (it->second.file_name != file) { - it->second.file_name = file; std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); + it->second.file_name = file; } } /// New file @@ -261,6 +257,8 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const void StorageFileLog::deserialize() { + if (!std::filesystem::exists(root_meta_path)) + return; /// In case of single file (not a watched directory), /// iterated directoy always has one file inside. for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) @@ -324,7 +322,7 @@ Pipe StorageFileLog::read( getStorageID().getTableName()); } - if (running_streams.load(std::memory_order_relaxed)) + if (running_streams) { throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } @@ -409,6 +407,9 @@ void StorageFileLog::shutdown() { task->stream_cancelled = true; + /// Reader thread may wait for wake up + wakeUp(); + LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } @@ -623,10 +624,13 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - std::unique_lock lock(mutex); - /// Waiting for watch directory thread to wake up + std::unique_lock lock(mutex); + /// Waiting for watch directory thread to wake up cv.wait(lock, [this] { return has_new_events; }); has_new_events = false; + + if (task->stream_cancelled) + return; task->holder->schedule(); } else @@ -636,7 +640,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - if (running_streams.load(std::memory_order_relaxed)) + if (running_streams) { throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } @@ -702,6 +706,14 @@ bool StorageFileLog::streamToViews() return updateFileInfos(); } +void StorageFileLog::wakeUp() +{ + std::unique_lock lock(mutex); + has_new_events = true; + lock.unlock(); + cv.notify_one(); +} + void registerStorageFileLog(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) @@ -767,7 +779,6 @@ void registerStorageFileLog(StorageFactory & factory) args.getContext(), args.columns, path, - args.relative_data_path, format, std::move(filelog_settings), args.comment, @@ -813,10 +824,10 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & [file_name, event_info] : events) + for (const auto & [file_name, event_infos] : events) { String file_path = getFullDataPath(file_name); - for(const auto & event_info : event_info.file_events) + for (const auto & event_info : event_infos.file_events) { switch (event_info.type) { @@ -836,7 +847,7 @@ bool StorageFileLog::updateFileInfos() file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - it->second = FileContext{.inode = inode}; + it->second = FileContext{.status = FileStatus::OPEN, .inode = inode}; else file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 04c83d8837d..da87cd1be5a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -126,9 +127,7 @@ public: void increaseStreams(); void reduceStreams(); - auto & getConditionVariable() { return cv; } - auto & getMutex() { return mutex; } - void setNewEvents() { has_new_events = true; } + void wakeUp(); const auto & getFileLogSettings() const { return filelog_settings; } @@ -138,7 +137,6 @@ protected: ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, const String & comment, @@ -148,14 +146,11 @@ private: std::unique_ptr filelog_settings; const String path; - /// For meta file - const String relative_data_path; bool path_is_directory = true; /// If path argument of the table is a regular file, it equals to user_files_path /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path String root_data_path; - /// relative_data_path/ + table_name/ String root_meta_path; FileInfos file_infos; @@ -163,20 +158,8 @@ private: const String format_name; Poco::Logger * log; - std::unique_ptr directory_watch = nullptr; - uint64_t milliseconds_to_wait; - struct TaskContext - { - BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled {false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) - { - } - }; - std::shared_ptr task; - /// In order to avoid data race, using a naive trick to forbid execute two select /// simultaneously, although read is not useful in this engine. Using an atomic /// variable to records current unfinishing streams, then if have unfinishing streams, @@ -189,6 +172,18 @@ private: bool has_new_events = false; std::condition_variable cv; + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::shared_ptr task; + + std::unique_ptr directory_watch = nullptr; + void loadFiles(); void loadMetaFiles(bool attach); diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference index 0ab71c65c6b..c787d2047db 100644 --- a/tests/queries/0_stateless/02023_storage_filelog.reference +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -141,4 +141,147 @@ 120 120 120 120 120 120 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 +110 110 +111 111 +112 112 +113 113 +114 114 +115 115 +116 116 +117 117 +118 118 +119 119 +120 120 +150 150 +151 151 +152 152 +153 153 +154 154 +155 155 +156 156 +157 157 +158 158 +159 159 +160 160 +161 161 +162 162 +163 163 +164 164 +165 165 +166 166 +167 167 +168 168 +169 169 +170 170 +171 171 +172 172 +173 173 +174 174 +175 175 +176 176 +177 177 +178 178 +179 179 +180 180 +181 181 +182 182 +183 183 +184 184 +185 185 +186 186 +187 187 +188 188 +189 189 +190 190 +191 191 +192 192 +193 193 +194 194 +195 195 +196 196 +197 197 +198 198 +199 199 +200 200 +200 200 +201 201 +202 202 +203 203 +204 204 +205 205 +206 206 +207 207 +208 208 +209 209 +210 210 +211 211 +212 212 +213 213 +214 214 +215 215 +216 216 +217 217 +218 218 +219 219 +220 220 +221 221 +222 222 +223 223 +224 224 +225 225 +226 226 +227 227 +228 228 +229 229 +230 230 +231 231 +232 232 +233 233 +234 234 +235 235 +236 236 +237 237 +238 238 +239 239 +240 240 +241 241 +242 242 +243 243 +244 244 +245 245 +246 246 +247 247 +248 248 +249 249 +250 250 OK diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index b695c270835..b13bb4da065 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p ${user_files_path}/logs/ @@ -41,11 +41,28 @@ touch ${user_files_path}/logs/a.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt +mv ${user_files_path}/logs/b.txt ${user_files_path}/logs/j.txt rm ${user_files_path}/logs/d.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +cp ${user_files_path}/logs/e.txt ${user_files_path}/logs/f.txt +mv ${user_files_path}/logs/e.txt ${user_files_path}/logs/g.txt +mv ${user_files_path}/logs/c.txt ${user_files_path}/logs/h.txt +for i in {150..200} +do + echo $i, $i >> ${user_files_path}/logs/h.txt +done +for i in {200..250} +do + echo $i, $i >> ${user_files_path}/logs/i.txt +done +${CLICKHOUSE_CLIENT} --query "attach table file_log;" + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + ${CLICKHOUSE_CLIENT} --query "detach table file_log;" ${CLICKHOUSE_CLIENT} --query "attach table file_log;" diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 0503552b8dd..95dfb74ecf6 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -14,10 +14,6 @@ user_files_path=$(clickhouse-client --query "select _path,_file from file('nonex mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* -for i in {1..20} -do - echo $i, $i >> ${user_files_path}/logs/a.txt -done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" @@ -25,6 +21,11 @@ ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=Fil ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + for i in {1..200} do sleep 0.1 From a32abda322c6cf3728d9529721f8ccd9067843a6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 07:08:37 +0000 Subject: [PATCH 793/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 21 ++++++++++++------- .../0_stateless/02024_storage_filelog_mv.sh | 10 ++++----- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 81303f623aa..f807663dbec 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -624,14 +624,19 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - std::unique_lock lock(mutex); - /// Waiting for watch directory thread to wake up - cv.wait(lock, [this] { return has_new_events; }); - has_new_events = false; + if (milliseconds_to_wait < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + task->holder->scheduleAfter(milliseconds_to_wait); + else + { + std::unique_lock lock(mutex); + /// Waiting for watch directory thread to wake up + cv.wait(lock, [this] { return has_new_events; }); + has_new_events = false; - if (task->stream_cancelled) - return; - task->holder->schedule(); + if (task->stream_cancelled) + return; + task->holder->schedule(); + } } else task->holder->scheduleAfter(milliseconds_to_wait); @@ -711,7 +716,7 @@ void StorageFileLog::wakeUp() std::unique_lock lock(mutex); has_new_events = true; lock.unlock(); - cv.notify_one(); + cv.notify_one(); } void registerStorageFileLog(StorageFactory & factory) diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 95dfb74ecf6..5170906b8d9 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -15,17 +15,17 @@ user_files_path=$(clickhouse-client --query "select _path,_file from file('nonex mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" -for i in {1..20} -do - echo $i, $i >> ${user_files_path}/logs/a.txt -done - for i in {1..200} do sleep 0.1 From 6cf82f859cd77377dfdc737b480000350066fee8 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 07:27:37 +0000 Subject: [PATCH 794/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 19 +++++++++++++------ src/Storages/FileLog/StorageFileLog.h | 2 ++ .../0_stateless/02023_storage_filelog.sh | 2 +- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index f807663dbec..1323331639d 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -569,14 +569,22 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) return true; } +size_t StorageFileLog::getTableDependentCount() const +{ + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + return DatabaseCatalog::instance().getDependencies(table_id).size(); +} + void StorageFileLog::threadFunc() { try { updateFileInfos(); + auto table_id = getStorageID(); - // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + + auto dependencies_count = getTableDependentCount(); if (dependencies_count) { @@ -593,9 +601,8 @@ void StorageFileLog::threadFunc() if (streamToViews()) { LOG_TRACE(log, "Stream stalled. Reschedule."); - if (!path_is_directory - && milliseconds_to_wait - < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + if (milliseconds_to_wait + < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) milliseconds_to_wait *= filelog_settings->poll_directory_watch_events_backoff_factor.value; break; } @@ -624,7 +631,7 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - if (milliseconds_to_wait < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + if (!getTableDependentCount()) task->holder->scheduleAfter(milliseconds_to_wait); else { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index da87cd1be5a..f0aac1a7985 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -199,6 +199,8 @@ private: bool updateFileInfos(); + size_t getTableDependentCount() const; + /// Used in shutdown() void serialize() const; /// Used in FileSource closeFileAndStoreMeta(file_name); diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index b13bb4da065..8279e09ff65 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Data preparation. # Now we can get the user_files_path by use the table file function for trick. also we can get it by query as: # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" -user_files_path=$(${CLICKHOUSE_CLIENT} --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') mkdir -p ${user_files_path}/logs/ From 6d4af3bac1aaeb3c423ad96f19627c700c3f3170 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 18 Oct 2021 10:23:46 +0300 Subject: [PATCH 795/950] Move SquashingTransform to Interpreters (to fix split build) clickhouse_common_io requires clickhouse_core: ld.lld: error: undefined symbol: DB::blocksHaveEqualStructure(DB::Block const&, DB::Block const&) >>> referenced by SquashingTransform.cpp:92 (/src/ch/clickhouse/src/Common/SquashingTransform.cpp:92) >>> src/CMakeFiles/clickhouse_common_io.dir/Common/SquashingTransform.cpp.o:(void DB::SquashingTransform::append(DB::Block&&)) >>> referenced by SquashingTransform.cpp:92 (/src/ch/clickhouse/src/Common/SquashingTransform.cpp:92) >>> src/CMakeFiles/clickhouse_common_io.dir/Common/SquashingTransform.cpp.o:(void DB::SquashingTransform::append(DB::Block const&)) while clickhouse_core requires clickhouse_common_io: "clickhouse_core" of type SHARED_LIBRARY depends on "roaring" (weak) depends on "clickhouse_common_io" (weak) depends on "clickhouse_common_config" (weak) depends on "clickhouse_common_zookeeper" (weak) depends on "clickhouse_dictionaries_embedded" (weak) depends on "clickhouse_parsers" (weak) Follow-up for: #30247 (cc @KochetovNicolai) --- src/{Common => Interpreters}/SquashingTransform.cpp | 3 +-- src/{Common => Interpreters}/SquashingTransform.h | 0 src/Processors/Transforms/SquashingChunksTransform.h | 3 ++- src/Storages/MergeTree/MutateTask.cpp | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) rename src/{Common => Interpreters}/SquashingTransform.cpp (98%) rename src/{Common => Interpreters}/SquashingTransform.h (100%) diff --git a/src/Common/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp similarity index 98% rename from src/Common/SquashingTransform.cpp rename to src/Interpreters/SquashingTransform.cpp index 21fa25ed3af..54d9a1db25e 100644 --- a/src/Common/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -1,5 +1,4 @@ -#include -#include +#include namespace DB diff --git a/src/Common/SquashingTransform.h b/src/Interpreters/SquashingTransform.h similarity index 100% rename from src/Common/SquashingTransform.h rename to src/Interpreters/SquashingTransform.h diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 45a0f33d666..bf4a051891b 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -1,6 +1,7 @@ #pragma once + #include -#include +#include namespace DB { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index eed8cb3822c..7ca676b04bf 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -2,12 +2,12 @@ #include #include +#include +#include #include #include -#include #include #include -#include #include #include #include From ec6b7785015c45a69a9c4224413a19df0a0fe412 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 18 Oct 2021 10:29:57 +0300 Subject: [PATCH 796/950] Update 02051_symlinks_to_user_files.sh --- .../0_stateless/02051_symlinks_to_user_files.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index 53c50542b06..fe3073f9ff2 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -8,17 +8,18 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # See 01658_read_file_to_string_column.sh user_files_path=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') -mkdir -p "${user_files_path}/" -chmod 777 "${user_files_path}" +FILE_PATH="${user_files_path}/file/" +mkdir -p ${FILE_PATH} +chmod 777 ${FILE_PATH} -export FILE="test_symlink_${CLICKHOUSE_DATABASE}" +FILE="test_symlink_${CLICKHOUSE_DATABASE}" -symlink_path=${user_files_path}/${FILE} +symlink_path=${FILE_PATH}/${FILE} file_path=$CUR_DIR/${FILE} touch ${file_path} -chmod +w ${file_path} ln -s ${file_path} ${symlink_path} +chmod +w ${symlink_path} function cleanup() { From 09b782a52e8f0a558ee5a75474825a6474dde760 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 18 Oct 2021 10:39:52 +0300 Subject: [PATCH 797/950] Increase default wait of the server start in clickhouse-test Set --server-check-retries to 90 (and this is ~45 seconds), since right now sometimes it is not enough [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/30191/0e34a9d550cfe6924fe575871f36c44dd44acdaa/functional_stateless_tests_(memory).html#fail1 And the reason I guess is clickhouse-test had been rewritten to http.client in #30065, and since now it does not need to execute clickhouse-client binary, which in debug/sanitizers builds can take also sometime. That said that with clickhouse-client for hung check it was not 15 seconds, but more (each clickhouse-client requires 0.6sec with sanitizers for simple SELECT 1, while w/o 0.1second, also too much should be optimized) --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1e8c2bfb8ad..5f6960e57c4 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1309,7 +1309,7 @@ if __name__ == '__main__': parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel') parser.add_argument('--test-runs', default=1, nargs='?', type=int, help='Run each test many times (useful for e.g. flaky check)') parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context') - parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started') + parser.add_argument('-r', '--server-check-retries', default=90, type=int, help='Num of tries to execute SELECT 1 before tests started') parser.add_argument('--db-engine', help='Database engine name') parser.add_argument('--replicated-database', action='store_true', default=False, help='Run tests with Replicated database engine') parser.add_argument('--fast-tests-only', action='store_true', default=False, help='Run only fast tests (the tests without the "no-fasttest" tag)') From 0c628726af1894c1f78ba3db4961a86c53ef3adb Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 07:48:58 +0000 Subject: [PATCH 798/950] fix build --- src/Storages/FileLog/FileLogSource.cpp | 1 - src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index bb0fcc90016..f046e745046 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 1323331639d..2a6e124e6e8 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -13,19 +13,19 @@ #include #include #include -#include +#include #include #include #include #include #include +#include #include #include #include #include #include #include -#include #include From 81fb4bcf76f995e70cdb0f062ffc6e4366210930 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 08:14:54 +0000 Subject: [PATCH 799/950] fix style --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 8 ++++---- src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index b5d914f700d..c459079ec06 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -118,10 +118,10 @@ void DirectoryWatcherBase::watchFunc() /// Wake up reader thread owner.storage.wakeUp(); } - else - { - if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) - milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; + else + { + if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; } } } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 2a6e124e6e8..92c30a86b49 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -260,7 +260,7 @@ void StorageFileLog::deserialize() if (!std::filesystem::exists(root_meta_path)) return; /// In case of single file (not a watched directory), - /// iterated directoy always has one file inside. + /// iterated directory always has one file inside. for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) { if (!dir_entry.is_regular_file()) @@ -317,7 +317,7 @@ Pipe StorageFileLog::read( { throw Exception( ErrorCodes::QUERY_NOT_ALLOWED, - "Can not make `SELECT` query from table {}, because it has attached dependencies. Remove dependant materialized views if " + "Can not make `SELECT` query from table {}, because it has attached dependencies. Remove dependent materialized views if " "needed", getStorageID().getTableName()); } From 109e664188fb658697282d9919b09ca05d90bd62 Mon Sep 17 00:00:00 2001 From: lhuang0928 Date: Mon, 18 Oct 2021 08:53:25 +0000 Subject: [PATCH 800/950] fix date32 comparison with datetime/datetime64 --- src/DataTypes/getLeastSupertype.cpp | 12 +++++++++--- src/Functions/FunctionsComparison.h | 9 +++++---- .../0_stateless/02098_date32_comparison.reference | 2 ++ .../queries/0_stateless/02098_date32_comparison.sql | 2 ++ 4 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index b77fcdcdfca..f8d10535be2 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -277,7 +277,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) /// For Date and DateTime/DateTime64, the common type is DateTime/DateTime64. No other types are compatible. { UInt32 have_date = type_ids.count(TypeIndex::Date); - UInt32 have_date32 = type_ids.count(TypeIndex::Date32); + UInt32 have_date32 = type_ids.count(TypeIndex::Date32); UInt32 have_datetime = type_ids.count(TypeIndex::DateTime); UInt32 have_datetime64 = type_ids.count(TypeIndex::DateTime64); @@ -299,8 +299,8 @@ DataTypePtr getLeastSupertype(const DataTypes & types) return std::make_shared(); } - /// For Date and Date32, the common type is Date32 - if (have_datetime == 0 && have_datetime64 == 0) + /// For Date and Date32, the common type is Date32 + if (have_datetime == 0 && have_datetime64 == 0) { for (const auto & type : types) { @@ -309,6 +309,12 @@ DataTypePtr getLeastSupertype(const DataTypes & types) } } + /// For Datetime and Date32, the common type is Datetime64 + if (have_datetime == 1 && have_date32 == 1 && have_datetime64 == 0) + { + return std::make_shared(0); + } + UInt8 max_scale = 0; size_t max_scale_date_time_index = 0; diff --git a/src/Functions/FunctionsComparison.h b/src/Functions/FunctionsComparison.h index 9b94ac589a3..945090781dc 100644 --- a/src/Functions/FunctionsComparison.h +++ b/src/Functions/FunctionsComparison.h @@ -1222,8 +1222,8 @@ public: } else if ((isColumnedAsDecimal(left_type) || isColumnedAsDecimal(right_type))) { - // Comparing Date and DateTime64 requires implicit conversion, - if (date_and_datetime && (isDate(left_type) || isDate(right_type))) + // Comparing Date/Date32 and DateTime64 requires implicit conversion, + if (date_and_datetime && (isDateOrDate32(left_type) || isDateOrDate32(right_type))) { DataTypePtr common_type = getLeastSupertype({left_type, right_type}); ColumnPtr c0_converted = castColumn(col_with_type_and_name_left, common_type); @@ -1247,9 +1247,10 @@ public: ColumnPtr c0_converted = castColumn(col_with_type_and_name_left, common_type); ColumnPtr c1_converted = castColumn(col_with_type_and_name_right, common_type); if (!((res = executeNumLeftType(c0_converted.get(), c1_converted.get())) + || (res = executeNumLeftType(c0_converted.get(), c1_converted.get())) || (res = executeNumLeftType(c0_converted.get(), c1_converted.get())) - || (res = executeNumLeftType(c0_converted.get(), c1_converted.get())))) - throw Exception("Date related common types can only be UInt32 or UInt64", ErrorCodes::LOGICAL_ERROR); + || (res = executeDecimal({c0_converted, common_type, "left"}, {c1_converted, common_type, "right"})))) + throw Exception("Date related common types can only be UInt32/UInt64/Int32/Decimal", ErrorCodes::LOGICAL_ERROR); return res; } else if (left_type->equals(*right_type)) diff --git a/tests/queries/0_stateless/02098_date32_comparison.reference b/tests/queries/0_stateless/02098_date32_comparison.reference index a9e2f17562a..9e1f26e6d5e 100644 --- a/tests/queries/0_stateless/02098_date32_comparison.reference +++ b/tests/queries/0_stateless/02098_date32_comparison.reference @@ -4,3 +4,5 @@ 1 1 1 +1 +1 \ No newline at end of file diff --git a/tests/queries/0_stateless/02098_date32_comparison.sql b/tests/queries/0_stateless/02098_date32_comparison.sql index d0a632977c4..dd4fde790c7 100644 --- a/tests/queries/0_stateless/02098_date32_comparison.sql +++ b/tests/queries/0_stateless/02098_date32_comparison.sql @@ -2,5 +2,7 @@ select toDate32('1990-01-01') = toDate('1990-01-01'); select toDate('1991-01-02') > toDate32('1990-01-01'); select toDate32('1990-01-01') = toDateTime('1990-01-01'); select toDateTime('1991-01-02') > toDate32('1990-01-01'); +select toDate32('1990-01-01') = toDateTime64('1990-01-01',2); +select toDateTime64('1991-01-02',2) > toDate32('1990-01-01'); select toDate32('1990-01-01') = '1990-01-01'; select '1991-01-02' > toDate32('1990-01-01'); \ No newline at end of file From 6d24ca4c3ab3b52b4ef1de7d79064d23f4af45bb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 18 Oct 2021 12:13:24 +0300 Subject: [PATCH 801/950] Review fixes --- programs/keeper/Keeper.cpp | 2 +- programs/server/Server.cpp | 14 +++- src/Interpreters/Context.cpp | 65 +++++++++++++++++-- src/Interpreters/Context.h | 10 +-- .../test_keeper_three_nodes_two_alive/test.py | 6 ++ 5 files changed, 86 insertions(+), 11 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index a3034150219..da3c42a3213 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -358,7 +358,7 @@ int Keeper::main(const std::vector & /*args*/) auto servers = std::make_shared>(); - /// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config. + /// Initialize keeper RAFT. Do nothing if no keeper_server in config. global_context->initializeKeeperDispatcher(/* start_async = */false); for (const auto & listen_host : listen_hosts) { diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fac039f2454..22a529df1e0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -997,9 +997,19 @@ if (ThreadFuzzer::instance().isEffective()) if (config().has("keeper_server")) { #if USE_NURAFT - bool has_connection = has_zookeeper && global_context->tryCheckZooKeeperConnection(); + //// If we don't have configured connection probably someone trying to use clickhouse-server instead + //// of clickhouse-keeper, so start synchronously. + bool can_initialize_keeper_async = false; + + if (has_zookeeper) /// We have configured connection to some zookeeper cluster + { + /// If we cannot connect to some other node from our cluster then we have to wait our Keeper start + /// synchronously. + can_initialize_keeper_async = global_context->tryCheckClientConnectionToMyKeeperCluster(); + } /// Initialize keeper RAFT. - global_context->initializeKeeperDispatcher(has_connection); + global_context->initializeKeeperDispatcher(can_initialize_keeper_async); + for (const auto & listen_host : listen_hosts) { /// TCP Keeper diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7ae531a6266..a61c2669ef2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1806,12 +1806,60 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const return shared->zookeeper; } +namespace +{ -bool Context::tryCheckZooKeeperConnection() const +bool checkZooKeeperConfigIsLocal(const Poco::Util::AbstractConfiguration & config, const std::string & config_name) +{ + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_name, keys); + + for (const auto & key : keys) + { + if (startsWith(key, "node")) + { + String host = config.getString(config_name + "." + key + ".host"); + if (isLocalAddress(DNSResolver::instance().resolveHost(host))) + return true; + } + } + return false; +} + +} + + +bool Context::tryCheckClientConnectionToMyKeeperCluster() const { try { - getZooKeeper(); + /// If our server is part of main Keeper cluster + if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper")) + { + LOG_DEBUG(shared->log, "Keeper server is participant of the main zookeeper cluster, will try to connect to it"); + getZooKeeper(); + /// Connected, return true + return true; + } + else + { + Poco::Util::AbstractConfiguration::Keys keys; + getConfigRef().keys("auxiliary_zookeepers", keys); + + /// If our server is part of some auxiliary_zookeeper + for (const auto & aux_zk_name : keys) + { + if (checkZooKeeperConfigIsLocal(getConfigRef(), "auxiliary_zookeepers." + aux_zk_name)) + { + LOG_DEBUG(shared->log, "Our Keeper server is participant of the auxiliary zookeeper cluster ({}), will try to connect to it", aux_zk_name); + getAuxiliaryZooKeeper(aux_zk_name); + /// Connected, return true + return true; + } + } + } + + /// Our server doesn't depend on our Keeper cluster return true; } catch (...) @@ -1860,8 +1908,17 @@ void Context::initializeKeeperDispatcher(bool start_async) const if (config.has("keeper_server")) { bool is_standalone_app = getApplicationType() == ApplicationType::KEEPER; - if (start_async && !is_standalone_app) - LOG_INFO(shared->log, "Connected to ZooKeeper (or Keeper) before internal Keeper start, will wait for Keeper asynchronously"); + if (start_async) + { + assert(!is_standalone_app); + LOG_INFO(shared->log, "Connected to ZooKeeper (or Keeper) before internal Keeper start or we don't depend on our Keeper cluster" + ", will wait for Keeper asynchronously"); + } + else + { + LOG_INFO(shared->log, "Cannot connect to ZooKeeper (or Keeper) before internal Keeper start," + "will wait for Keeper synchronously"); + } shared->keeper_storage_dispatcher = std::make_shared(); shared->keeper_storage_dispatcher->initialize(config, is_standalone_app, start_async); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index dc74c4ac49a..5a28e3fac97 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -13,6 +13,7 @@ #include #include #include +#include #include #if !defined(ARCADIA_BUILD) @@ -664,10 +665,11 @@ public: /// Same as above but return a zookeeper connection from auxiliary_zookeepers configuration entry. std::shared_ptr getAuxiliaryZooKeeper(const String & name) const; - /// Try to connect to ZooKeeper using getZooKeeper. Useful for internal - /// Keeper start (check connection to some other node). Return true if - /// connected successfully (without exception). - bool tryCheckZooKeeperConnection() const; + /// Try to connect to Keeper using get(Auxiliary)ZooKeeper. Useful for + /// internal Keeper start (check connection to some other node). Return true + /// if connected successfully (without exception) or our zookeeper client + /// connection configured for some other cluster without our node. + bool tryCheckClientConnectionToMyKeeperCluster() const; UInt32 getZooKeeperSessionUptime() const; diff --git a/tests/integration/test_keeper_three_nodes_two_alive/test.py b/tests/integration/test_keeper_three_nodes_two_alive/test.py index 2c13d3ef22b..eb63d28b3e2 100644 --- a/tests/integration/test_keeper_three_nodes_two_alive/test.py +++ b/tests/integration/test_keeper_three_nodes_two_alive/test.py @@ -49,6 +49,9 @@ def test_start_offline(started_cluster): time.sleep(3) p.map(start, [node2, node3]) + assert node2.contains_in_log("Cannot connect to ZooKeeper (or Keeper) before internal Keeper start") + assert node3.contains_in_log("Cannot connect to ZooKeeper (or Keeper) before internal Keeper start") + node2_zk = get_fake_zk("node2") node2_zk.create("/test_dead", b"data") finally: @@ -68,6 +71,9 @@ def test_start_non_existing(started_cluster): time.sleep(3) p.map(start, [node2, node1]) + assert node1.contains_in_log("Cannot connect to ZooKeeper (or Keeper) before internal Keeper start") + assert node2.contains_in_log("Cannot connect to ZooKeeper (or Keeper) before internal Keeper start") + node2_zk = get_fake_zk("node2") node2_zk.create("/test_non_exising", b"data") finally: From 1131a3ebe8056071b6a3aad19f88e53d90cc5ab9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 12:15:08 +0300 Subject: [PATCH 802/950] fix test --- .../0_stateless/01604_explain_ast_of_nonselect_query.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference b/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference index 3545278ec1a..3c92ffb0a8c 100644 --- a/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference +++ b/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference @@ -1,6 +1,6 @@ AlterQuery t1 (children 1) ExpressionList (children 1) - AlterCommand 32 (children 1) + AlterCommand 33 (children 1) Function equals (children 1) ExpressionList (children 2) Identifier date From e543f720c696cc22169256492807aee9dd8d287f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 12:29:34 +0300 Subject: [PATCH 803/950] fix clang-tidy --- .../Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 29209f462e4..8b83e4e215b 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -40,7 +40,7 @@ public: AggregatingTransformParamsPtr params_, SortDescription description_, size_t max_block_size_, - size_t max_threads_); + size_t merge_threads_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; From 6e479b301a3052d691e81770fe4a076ca5718479 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 12:54:12 +0300 Subject: [PATCH 804/950] Update memory optimisation for MergingSorted. --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +--- src/Processors/Merges/AggregatingSortedTransform.h | 2 +- src/Processors/Merges/CollapsingSortedTransform.h | 2 +- .../Merges/FinishAggregatingInOrderTransform.h | 2 +- src/Processors/Merges/GraphiteRollupSortedTransform.h | 2 +- src/Processors/Merges/IMergingTransform.cpp | 9 ++++++--- src/Processors/Merges/IMergingTransform.h | 8 ++++---- src/Processors/Merges/MergingSortedTransform.cpp | 3 +-- src/Processors/Merges/MergingSortedTransform.h | 1 - src/Processors/Merges/ReplacingSortedTransform.h | 2 +- src/Processors/Merges/SummingSortedTransform.h | 2 +- src/Processors/Merges/VersionedCollapsingTransform.h | 2 +- src/Processors/QueryPlan/FinishSortingStep.cpp | 8 ++------ src/Processors/QueryPlan/FinishSortingStep.h | 4 +--- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 3 +-- src/Processors/Transforms/ColumnGathererTransform.cpp | 2 +- src/Processors/Transforms/MergeSortingTransform.cpp | 1 - .../tests/gtest_blocks_size_merging_streams.cpp | 4 ++-- src/Storages/MergeTree/MergeTask.cpp | 2 +- 19 files changed, 27 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5dbde2344d3..35c8c32c65b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2303,14 +2303,12 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input { const Settings & settings = context->getSettingsRef(); - const auto & query = getSelectQuery(); auto finish_sorting_step = std::make_unique( query_plan.getCurrentDataStream(), input_sorting_info->order_key_prefix_descr, output_order_descr, settings.max_block_size, - limit, - query.hasFiltration()); + limit); query_plan.addStep(std::move(finish_sorting_step)); } diff --git a/src/Processors/Merges/AggregatingSortedTransform.h b/src/Processors/Merges/AggregatingSortedTransform.h index e8bf90c2b31..b0cdf4c8a3c 100644 --- a/src/Processors/Merges/AggregatingSortedTransform.h +++ b/src/Processors/Merges/AggregatingSortedTransform.h @@ -16,7 +16,7 @@ public: const Block & header, size_t num_inputs, SortDescription description_, size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index 87c466f31e8..a37e1c8402f 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -20,7 +20,7 @@ public: WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 6d5e334311f..58bd399afda 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -19,7 +19,7 @@ public: SortDescription description, size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, params, diff --git a/src/Processors/Merges/GraphiteRollupSortedTransform.h b/src/Processors/Merges/GraphiteRollupSortedTransform.h index 46272f00eed..e6307c629ea 100644 --- a/src/Processors/Merges/GraphiteRollupSortedTransform.h +++ b/src/Processors/Merges/GraphiteRollupSortedTransform.h @@ -15,7 +15,7 @@ public: SortDescription description_, size_t max_block_size, Graphite::Params params_, time_t time_of_merge_) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index cba78390c97..2e0e21b7fe0 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -15,10 +15,10 @@ IMergingTransformBase::IMergingTransformBase( const Block & input_header, const Block & output_header, bool have_all_inputs_, - bool has_limit_below_one_block_) + size_t limit_hint_) : IProcessor(InputPorts(num_inputs, input_header), {output_header}) , have_all_inputs(have_all_inputs_) - , has_limit_below_one_block(has_limit_below_one_block_) + , limit_hint(limit_hint_) { } @@ -79,7 +79,10 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs() /// setNotNeeded after reading first chunk, because in optimismtic case /// (e.g. with optimized 'ORDER BY primary_key LIMIT n' and small 'n') /// we won't have to read any chunks anymore; - auto chunk = input.pull(has_limit_below_one_block); + auto chunk = input.pull(limit_hint != 0); + if (limit_hint && chunk.getNumRows() < limit_hint) + input.setNeeded(); + if (!chunk.hasRows()) { if (!input.isFinished()) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 8b0a44ae025..155b705450b 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -17,7 +17,7 @@ public: const Block & input_header, const Block & output_header, bool have_all_inputs_, - bool has_limit_below_one_block_); + size_t limit_hint_); OutputPort & getOutputPort() { return outputs.front(); } @@ -67,7 +67,7 @@ private: std::vector input_states; std::atomic have_all_inputs; bool is_initialized = false; - bool has_limit_below_one_block = false; + size_t limit_hint = 0; IProcessor::Status prepareInitializeInputs(); }; @@ -83,9 +83,9 @@ public: const Block & input_header, const Block & output_header, bool have_all_inputs_, - bool has_limit_below_one_block_, + size_t limit_hint_, Args && ... args) - : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, has_limit_below_one_block_) + : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, limit_hint_) , algorithm(std::forward(args) ...) { } diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 667972e3cf6..ec864b561e9 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -13,13 +13,12 @@ MergingSortedTransform::MergingSortedTransform( SortDescription description_, size_t max_block_size, UInt64 limit_, - bool has_limit_below_one_block_, WriteBuffer * out_row_sources_buf_, bool quiet_, bool use_average_block_sizes, bool have_all_inputs_) : IMergingTransform( - num_inputs, header, header, have_all_inputs_, has_limit_below_one_block_, + num_inputs, header, header, have_all_inputs_, limit_, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/MergingSortedTransform.h b/src/Processors/Merges/MergingSortedTransform.h index 1fa9b1275bd..93bd36d8aec 100644 --- a/src/Processors/Merges/MergingSortedTransform.h +++ b/src/Processors/Merges/MergingSortedTransform.h @@ -17,7 +17,6 @@ public: SortDescription description, size_t max_block_size, UInt64 limit_ = 0, - bool has_limit_below_one_block_ = false, WriteBuffer * out_row_sources_buf_ = nullptr, bool quiet_ = false, bool use_average_block_sizes = false, diff --git a/src/Processors/Merges/ReplacingSortedTransform.h b/src/Processors/Merges/ReplacingSortedTransform.h index e760cdf0d2b..dfb386684fc 100644 --- a/src/Processors/Merges/ReplacingSortedTransform.h +++ b/src/Processors/Merges/ReplacingSortedTransform.h @@ -18,7 +18,7 @@ public: WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 0287caed5aa..0530ac2e96b 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -19,7 +19,7 @@ public: const Names & partition_key_columns, size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, std::move(description_), diff --git a/src/Processors/Merges/VersionedCollapsingTransform.h b/src/Processors/Merges/VersionedCollapsingTransform.h index f260e20f1da..5eced1cb58d 100644 --- a/src/Processors/Merges/VersionedCollapsingTransform.h +++ b/src/Processors/Merges/VersionedCollapsingTransform.h @@ -19,7 +19,7 @@ public: WriteBuffer * out_row_sources_buf_ = nullptr, bool use_average_block_sizes = false) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, header, num_inputs, std::move(description_), diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index 6347b69901c..9002c804e7b 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -31,14 +31,12 @@ FinishSortingStep::FinishSortingStep( SortDescription prefix_description_, SortDescription result_description_, size_t max_block_size_, - UInt64 limit_, - bool has_filtration_) + UInt64 limit_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(limit_)) , prefix_description(std::move(prefix_description_)) , result_description(std::move(result_description_)) , max_block_size(max_block_size_) , limit(limit_) - , has_filtration(has_filtration_) { /// TODO: check input_stream is sorted by prefix_description. output_stream->sort_description = result_description; @@ -60,14 +58,12 @@ void FinishSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const if (pipeline.getNumStreams() > 1) { UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - bool has_limit_below_one_block = !has_filtration && limit_for_merging && limit_for_merging < max_block_size; auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), prefix_description, max_block_size, - limit_for_merging, - has_limit_below_one_block); + limit_for_merging); pipeline.addTransform(std::move(transform)); } diff --git a/src/Processors/QueryPlan/FinishSortingStep.h b/src/Processors/QueryPlan/FinishSortingStep.h index ac34aea9df4..fd56c4353e7 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.h +++ b/src/Processors/QueryPlan/FinishSortingStep.h @@ -14,8 +14,7 @@ public: SortDescription prefix_description_, SortDescription result_description_, size_t max_block_size_, - UInt64 limit_, - bool has_filtration_); + UInt64 limit_); String getName() const override { return "FinishSorting"; } @@ -32,7 +31,6 @@ private: SortDescription result_description; size_t max_block_size; UInt64 limit; - bool has_filtration; }; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index cc400aacf2a..57785a5cc2d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -485,8 +485,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( pipe.getHeader(), pipe.numOutputPorts(), sort_description, - max_block_size, - 0, true); + max_block_size); pipe.addTransform(std::move(transform)); } diff --git a/src/Processors/Transforms/ColumnGathererTransform.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp index ddb8a5a0d68..d0cb4975290 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -127,7 +127,7 @@ ColumnGathererTransform::ColumnGathererTransform( ReadBuffer & row_sources_buf_, size_t block_preferred_size_) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, num_inputs, row_sources_buf_, block_preferred_size_) , log(&Poco::Logger::get("ColumnGathererStream")) { diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index e272fd0f183..73817d7de4a 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -197,7 +197,6 @@ void MergeSortingTransform::consume(Chunk chunk) description, max_merged_block_size, limit, - false, nullptr, quiet, use_average_block_sizes, diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index d39c49bb61c..fb10601216e 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -83,7 +83,7 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - DEFAULT_MERGE_BLOCK_SIZE, 0, false, nullptr, false, true); + DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); pipe.addTransform(std::move(transform)); @@ -130,7 +130,7 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) EXPECT_EQ(pipe.numOutputPorts(), 3); auto transform = std::make_shared(pipe.getHeader(), pipe.numOutputPorts(), sort_description, - DEFAULT_MERGE_BLOCK_SIZE, 0, false, nullptr, false, true); + DEFAULT_MERGE_BLOCK_SIZE, 0, nullptr, false, true); pipe.addTransform(std::move(transform)); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ea4821fc33d..5cb819c44a4 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -778,7 +778,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() { case MergeTreeData::MergingParams::Ordinary: merged_transform = std::make_shared( - header, pipes.size(), sort_description, merge_block_size, 0, false, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size); + header, pipes.size(), sort_description, merge_block_size, 0, ctx->rows_sources_write_buf.get(), true, ctx->blocks_are_granules_size); break; case MergeTreeData::MergingParams::Collapsing: From a6937a21cf1f8c22fef470f8fd3461527be6f88d Mon Sep 17 00:00:00 2001 From: lhuang0928 Date: Mon, 18 Oct 2021 09:58:53 +0000 Subject: [PATCH 805/950] fix a bug:reference file without a newline at the end --- tests/queries/0_stateless/02098_date32_comparison.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02098_date32_comparison.reference b/tests/queries/0_stateless/02098_date32_comparison.reference index 9e1f26e6d5e..c18b4e9b082 100644 --- a/tests/queries/0_stateless/02098_date32_comparison.reference +++ b/tests/queries/0_stateless/02098_date32_comparison.reference @@ -5,4 +5,4 @@ 1 1 1 -1 \ No newline at end of file +1 From 042e61febf9cf6ac95ed3acb0d830c0a772ad941 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 15 Oct 2021 13:53:25 +0300 Subject: [PATCH 806/950] Allow identifiers staring with numbers in multiple joins --- .../JoinToSubqueryTransformVisitor.cpp | 5 ++-- .../01120_join_constants.reference | 2 ++ .../0_stateless/01120_join_constants.sql | 18 +++++++++++++ ...96_join_unusual_identifier_begin.reference | 2 ++ .../02096_join_unusual_identifier_begin.sql | 27 +++++++++++++++++++ 5 files changed, 52 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02096_join_unusual_identifier_begin.reference create mode 100644 tests/queries/0_stateless/02096_join_unusual_identifier_begin.sql diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 3cd076f91bb..ed20b1b2048 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -18,7 +18,7 @@ #include #include #include - +#include namespace DB { @@ -524,7 +524,8 @@ std::vector normalizeColumnNamesExtractNeeded( size_t count = countTablesWithColumn(tables, short_name); - if (count > 1 || aliases.count(short_name)) + /// isValidIdentifierBegin retuired to be consistent with TableJoin::deduplicateAndQualifyColumnNames + if (count > 1 || aliases.count(short_name) || !isValidIdentifierBegin(short_name.at(0))) { const auto & table = tables[*table_pos]; IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column diff --git a/tests/queries/0_stateless/01120_join_constants.reference b/tests/queries/0_stateless/01120_join_constants.reference index a16427fbdf7..91838e7a2bb 100644 --- a/tests/queries/0_stateless/01120_join_constants.reference +++ b/tests/queries/0_stateless/01120_join_constants.reference @@ -1,2 +1,4 @@ 1 hello 1 world world 1 2 hello 0 world 1 +1 321 1 123 123 1 +2 321 0 0 123 1 diff --git a/tests/queries/0_stateless/01120_join_constants.sql b/tests/queries/0_stateless/01120_join_constants.sql index 443559c3ea1..d6d6a1be43b 100644 --- a/tests/queries/0_stateless/01120_join_constants.sql +++ b/tests/queries/0_stateless/01120_join_constants.sql @@ -15,3 +15,21 @@ LEFT JOIN arrayJoin([1, 3]) AS k, 'world' ) AS t2 ON t1.k = t2.k; + +SELECT + t1.*, + t2.*, + 123, + isConstant('world') +FROM +( + SELECT + arrayJoin([1, 2]) AS k, + 321 +) AS t1 +LEFT JOIN +( + SELECT + arrayJoin([1, 3]) AS k, + 123 +) AS t2 ON t1.k = t2.k; diff --git a/tests/queries/0_stateless/02096_join_unusual_identifier_begin.reference b/tests/queries/0_stateless/02096_join_unusual_identifier_begin.reference new file mode 100644 index 00000000000..e8cc5e526c0 --- /dev/null +++ b/tests/queries/0_stateless/02096_join_unusual_identifier_begin.reference @@ -0,0 +1,2 @@ +1 1 1 1 1 1 +1 diff --git a/tests/queries/0_stateless/02096_join_unusual_identifier_begin.sql b/tests/queries/0_stateless/02096_join_unusual_identifier_begin.sql new file mode 100644 index 00000000000..fc6be2eff7b --- /dev/null +++ b/tests/queries/0_stateless/02096_join_unusual_identifier_begin.sql @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; + +CREATE TABLE t1 ( `a1` Int64, `1a1` Int64 ) ENGINE = Memory; +INSERT INTO t1 VALUES (1, 1); + +CREATE TABLE t2 ( `b1` Int64, `1b1` Int64 ) ENGINE = Memory; +INSERT INTO t2 VALUES (1, 1); + +CREATE TABLE t3 ( `c1` Int64, `1c1` Int64 ) ENGINE = Memory; +INSERT INTO t3 VALUES (1, 1); + +SELECT + * +FROM t1 AS t1 +INNER JOIN t2 AS t2 ON t1.a1 = t2.b1 +INNER JOIN t3 AS t3 ON t1.a1 = t3.c1; + +SELECT t2.`1b1` FROM t1 JOIN t2 ON a1 = b1; + +-- Without quialification it doesn't work: +-- SELECT `1b1` FROM t1 JOIN t2 ON a1 = b1; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; From a28b048415bb330d5427a25d982378d425eba57e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 18 Oct 2021 14:22:27 +0300 Subject: [PATCH 807/950] Fix test --- .../0_stateless/01339_client_unrecognized_option.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01339_client_unrecognized_option.sh b/tests/queries/0_stateless/01339_client_unrecognized_option.sh index f88d890f33c..00c153ec915 100755 --- a/tests/queries/0_stateless/01339_client_unrecognized_option.sh +++ b/tests/queries/0_stateless/01339_client_unrecognized_option.sh @@ -5,14 +5,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT xyzgarbage 2>&1 | grep -q "Code: 552" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT xyzgarbage 2>&1 | grep -q "BAD_ARGUMENTS" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -xyzgarbage 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -xyzgarbage 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT --xyzgarbage 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT --xyzgarbage 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo 'OK' || echo 'FAIL' -cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' xyzgarbage 2>&1 | grep -q "Code: 552" && echo 'OK' || echo 'FAIL' +cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' xyzgarbage 2>&1 | grep -q "BAD_ARGUMENTS" && echo 'OK' || echo 'FAIL' -cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external -xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' +cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external -xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo 'OK' || echo 'FAIL' -cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "Bad arguments" && echo 'OK' || echo 'FAIL' +cat /etc/passwd | sed 's/:/\t/g' | $CLICKHOUSE_CLIENT --query="SELECT shell, count() AS c FROM passwd GROUP BY shell ORDER BY c DESC" --external --xyzgarbage --file=- --name=passwd --structure='login String, unused String, uid UInt16, gid UInt16, comment String, home String, shell String' 2>&1 | grep -q "UNRECOGNIZED_ARGUMENTS" && echo 'OK' || echo 'FAIL' From 6350957709831641b3b7c43d97a641ca86677d50 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Oct 2021 14:30:02 +0300 Subject: [PATCH 808/950] Fix special build --- src/Formats/CapnProtoUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/CapnProtoUtils.cpp b/src/Formats/CapnProtoUtils.cpp index 1dc37ff51ec..ecfa5df8351 100644 --- a/src/Formats/CapnProtoUtils.cpp +++ b/src/Formats/CapnProtoUtils.cpp @@ -422,7 +422,7 @@ void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Blo getCapnProtoFullTypeName(field.getType())); if (!additional_error_message.empty()) e.addMessage(additional_error_message); - throw e; + throw std::move(e); } } } From b2976fadb065571124ca675c04314d98e51b7933 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 11:57:03 +0000 Subject: [PATCH 809/950] fix test --- .../0_stateless/02025_storage_filelog_virtual_col.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 781253c2184..a92f93991ef 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -24,11 +24,11 @@ done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" for i in {100..120} do @@ -44,13 +44,13 @@ cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt rm ${user_files_path}/logs/d.txt -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" truncate ${user_files_path}/logs/a.txt --size 0 From 3c8e9634205951e373cb25cfef963e5a41401903 Mon Sep 17 00:00:00 2001 From: michon470 <71978106+michon470@users.noreply.github.com> Date: Mon, 18 Oct 2021 15:40:26 +0300 Subject: [PATCH 810/950] DOCSUP-15198: output_format_csv_null_representation setting translation (#29977) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Перевод без изменений содержания * в ру-версию добавлены ngrambf_v1 и tokenbf_v1 * Update mergetree.md * Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Update docs/ru/engines/table-engines/mergetree-family/mergetree.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> * Corrections and translation * in EN ver. lines 349-351 were included into the codeblock -- moved them to the proper place * ... Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../mergetree-family/mergetree.md | 9 ++--- .../mergetree-family/mergetree.md | 17 +++++++-- docs/ru/operations/settings/settings.md | 37 +++++++++++++++++++ 3 files changed, 54 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 4f473279067..aeaf39e28cb 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -320,7 +320,7 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 - `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` - Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with strings. Can be used for optimization of `equals`, `like` and `in` expressions. + Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](../../../sql-reference/data-types/string.md), [FixedString](../../../sql-reference/data-types/fixedstring.md) and [Map](../../../sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions. - `n` — ngram size, - `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well). @@ -337,7 +337,9 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`, `Map`. - For `Map` data type client can specify if index should be created for keys or values using [mapKeys](../../../sql-reference/functions/tuple-map-functions.md#mapkeys) or [mapValues](../../../sql-reference/functions/tuple-map-functions.md#mapvalues) function. + For `Map` data type client can specify if index should be created for keys or values using [mapKeys](../../../sql-reference/functions/tuple-map-functions.md#mapkeys) or [mapValues](../../../sql-reference/functions/tuple-map-functions.md#mapvalues) function. + + The following functions can use the filter: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem). Example of index creation for `Map` data type @@ -346,9 +348,6 @@ INDEX map_key_index mapKeys(map_column) TYPE bloom_filter GRANULARITY 1 INDEX map_key_index mapValues(map_column) TYPE bloom_filter GRANULARITY 1 ``` - The following functions can use it: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md). - - ``` sql INDEX sample_index (u64 * length(s)) TYPE minmax GRANULARITY 4 diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index bef14924d36..07e67ad1b85 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -316,17 +316,26 @@ SELECT count() FROM table WHERE u64 * i32 == 10 AND u64 * length(s) >= 1234 #### Доступные индексы {#available-types-of-indices} -- `minmax` — Хранит минимум и максимум выражения (если выражение - `tuple`, то для каждого элемента `tuple`), используя их для пропуска блоков аналогично первичному ключу. +- `minmax` — хранит минимум и максимум выражения (если выражение - [Tuple](../../../sql-reference/data-types/tuple.md), то для каждого элемента `Tuple`), используя их для пропуска блоков аналогично первичному ключу. -- `set(max_rows)` — Хранит уникальные значения выражения на блоке в количестве не более `max_rows` (если `max_rows = 0`, то ограничений нет), используя их для пропуска блоков, оценивая выполнимость `WHERE` выражения на хранимых данных. +- `set(max_rows)` — хранит уникальные значения выражения на блоке в количестве не более `max_rows` (если `max_rows = 0`, то ограничений нет), используя их для пропуска блоков, оценивая выполнимость `WHERE` выражения на хранимых данных. + +- `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` — хранит [фильтр Блума](https://en.wikipedia.org/wiki/Bloom_filter), содержащий все N-граммы блока данных. Работает только с данными форматов [String](../../../sql-reference/data-types/string.md), [FixedString](../../../sql-reference/data-types/fixedstring.md) и [Map](../../../sql-reference/data-types/map.md) с ключами типа `String` или `fixedString`. Может быть использован для оптимизации выражений `EQUALS`, `LIKE` и `IN`. + + - `n` — размер N-граммы, + - `size_of_bloom_filter_in_bytes` — размер в байтах фильтра Блума (можно использовать большие значения, например, 256 или 512, поскольку сжатие компенсирует возможные издержки). + - `number_of_hash_functions` — количество хеш-функций, использующихся в фильтре Блума. + - `random_seed` — состояние генератора случайных чисел для хеш-функций фильтра Блума. + +- `tokenbf_v1(size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` — то же, что и`ngrambf_v1`, но хранит токены вместо N-грамм. Токены — это последовательности символов, разделенные не буквенно-цифровыми символами. - `bloom_filter([false_positive])` — [фильтр Блума](https://en.wikipedia.org/wiki/Bloom_filter) для указанных стоблцов. Необязательный параметр `false_positive` — это вероятность получения ложноположительного срабатывания. Возможные значения: (0, 1). Значение по умолчанию: 0.025. - Поддержанные типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`. + Поддерживаемые типы данных: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`. - Фильтром могут пользоваться функции: [equals](../../../engines/table-engines/mergetree-family/mergetree.md), [notEquals](../../../engines/table-engines/mergetree-family/mergetree.md), [in](../../../engines/table-engines/mergetree-family/mergetree.md), [notIn](../../../engines/table-engines/mergetree-family/mergetree.md). + Фильтром могут пользоваться функции: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions.md), [notIn](../../../sql-reference/functions/in-functions.md), [has](../../../sql-reference/functions/array-functions.md#hasarr-elem). **Примеры** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e6e697e5eb6..71b6ac513fc 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2684,6 +2684,43 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); Значение по умолчанию: `1`. +## output_format_csv_null_representation {#output_format_csv_null_representation} + +Определяет представление `NULL` для формата выходных данных [CSV](../../interfaces/formats.md#csv). Пользователь может установить в качестве значения любую строку, например, `My NULL`. + +Значение по умолчанию: `\N`. + +**Примеры** + +Запрос: + +```sql +SELECT * FROM csv_custom_null FORMAT CSV; +``` + +Результат: + +```text +788 +\N +\N +``` + +Запрос: + +```sql +SET output_format_csv_null_representation = 'My NULL'; +SELECT * FROM csv_custom_null FORMAT CSV; +``` + +Результат: + +```text +788 +My NULL +My NULL +``` + ## output_format_tsv_null_representation {#output_format_tsv_null_representation} Определяет представление `NULL` для формата выходных данных [TSV](../../interfaces/formats.md#tabseparated). Пользователь может установить в качестве значения любую строку. From 5a4a752cfbf3de81f17e407a3dfccaffa6b27e13 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 17:37:46 +0300 Subject: [PATCH 811/950] Unite sorting steps. --- src/Interpreters/InterpreterSelectQuery.cpp | 50 +--- src/Processors/QueryPlan/MergeSortingStep.cpp | 96 ------- src/Processors/QueryPlan/MergeSortingStep.h | 47 ---- .../QueryPlan/MergingSortedStep.cpp | 85 ------ src/Processors/QueryPlan/MergingSortedStep.h | 36 --- .../QueryPlan/PartialSortingStep.cpp | 93 ------- src/Processors/QueryPlan/PartialSortingStep.h | 35 --- src/Processors/QueryPlan/SortingStep.cpp | 241 ++++++++++++++++++ src/Processors/QueryPlan/SortingStep.h | 76 ++++++ 9 files changed, 327 insertions(+), 432 deletions(-) delete mode 100644 src/Processors/QueryPlan/MergeSortingStep.cpp delete mode 100644 src/Processors/QueryPlan/MergeSortingStep.h delete mode 100644 src/Processors/QueryPlan/MergingSortedStep.cpp delete mode 100644 src/Processors/QueryPlan/MergingSortedStep.h delete mode 100644 src/Processors/QueryPlan/PartialSortingStep.cpp delete mode 100644 src/Processors/QueryPlan/PartialSortingStep.h create mode 100644 src/Processors/QueryPlan/SortingStep.cpp create mode 100644 src/Processors/QueryPlan/SortingStep.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 35c8c32c65b..c6ef18d7660 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -43,15 +43,12 @@ #include #include #include -#include #include #include #include -#include +#include #include -#include #include -#include #include #include #include @@ -2260,35 +2257,20 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) // happens in case of `over ()`. if (!w.full_sort_description.empty() && (i == 0 || !sortIsPrefix(w, *windows_sorted[i - 1]))) { - auto partial_sorting = std::make_unique( - query_plan.getCurrentDataStream(), - w.full_sort_description, - 0 /* LIMIT */, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode)); - partial_sorting->setStepDescription("Sort each block for window '" + w.window_name + "'"); - query_plan.addStep(std::move(partial_sorting)); - auto merge_sorting_step = std::make_unique( + auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), w.full_sort_description, settings.max_block_size, 0 /* LIMIT */, + SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, settings.max_bytes_before_external_sort, context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data); - merge_sorting_step->setStepDescription("Merge sorted blocks for window '" + w.window_name + "'"); - query_plan.addStep(std::move(merge_sorting_step)); - - // First MergeSorted, now MergingSorted. - auto merging_sorted = std::make_unique( - query_plan.getCurrentDataStream(), - w.full_sort_description, - settings.max_block_size, - 0 /* LIMIT */); - merging_sorted->setStepDescription("Merge sorted streams for window '" + w.window_name + "'"); - query_plan.addStep(std::move(merging_sorted)); + sorting_step->setStepDescription("Sorting for window '" + w.window_name + "'"); + query_plan.addStep(std::move(sorting_step)); } auto window_step = std::make_unique(query_plan.getCurrentDataStream(), w, w.window_functions); @@ -2303,7 +2285,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input { const Settings & settings = context->getSettingsRef(); - auto finish_sorting_step = std::make_unique( + auto finish_sorting_step = std::make_unique( query_plan.getCurrentDataStream(), input_sorting_info->order_key_prefix_descr, output_order_descr, @@ -2333,32 +2315,20 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo const Settings & settings = context->getSettingsRef(); - auto partial_sorting = std::make_unique( - query_plan.getCurrentDataStream(), - output_order_descr, - limit, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode)); - - partial_sorting->setStepDescription("Sort each block for ORDER BY"); - query_plan.addStep(std::move(partial_sorting)); - /// Merge the sorted blocks. - auto merge_sorting_step = std::make_unique( + auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), output_order_descr, settings.max_block_size, limit, + SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), settings.max_bytes_before_remerge_sort, settings.remerge_sort_lowered_memory_bytes_ratio, settings.max_bytes_before_external_sort, context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data); - merge_sorting_step->setStepDescription("Merge sorted blocks for ORDER BY"); - query_plan.addStep(std::move(merge_sorting_step)); - - /// If there are several streams, we merge them into one - executeMergeSorted(query_plan, output_order_descr, limit, "for ORDER BY"); + sorting_step->setStepDescription("Sorting for ORDER BY"); } @@ -2376,7 +2346,7 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const So const Settings & settings = context->getSettingsRef(); auto merging_sorted - = std::make_unique(query_plan.getCurrentDataStream(), sort_description, settings.max_block_size, limit); + = std::make_unique(query_plan.getCurrentDataStream(), sort_description, settings.max_block_size, limit); merging_sorted->setStepDescription("Merge sorted streams " + description); query_plan.addStep(std::move(merging_sorted)); diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp deleted file mode 100644 index 534f05a4d6e..00000000000 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ /dev/null @@ -1,96 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = true, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -MergeSortingStep::MergeSortingStep( - const DataStream & input_stream, - const SortDescription & description_, - size_t max_merged_block_size_, - UInt64 limit_, - size_t max_bytes_before_remerge_, - double remerge_lowered_memory_bytes_ratio_, - size_t max_bytes_before_external_sort_, - VolumePtr tmp_volume_, - size_t min_free_disk_space_) - : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) - , description(description_) - , max_merged_block_size(max_merged_block_size_) - , limit(limit_) - , max_bytes_before_remerge(max_bytes_before_remerge_) - , remerge_lowered_memory_bytes_ratio(remerge_lowered_memory_bytes_ratio_) - , max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_volume(tmp_volume_) - , min_free_disk_space(min_free_disk_space_) -{ - /// TODO: check input_stream is partially sorted by the same description. - output_stream->sort_description = description; - output_stream->sort_mode = input_stream.has_single_port ? DataStream::SortMode::Stream - : DataStream::SortMode::Port; -} - -void MergeSortingStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void MergeSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return nullptr; - - return std::make_shared( - header, description, max_merged_block_size, limit, - max_bytes_before_remerge / pipeline.getNumStreams(), - remerge_lowered_memory_bytes_ratio, - max_bytes_before_external_sort, - tmp_volume, - min_free_disk_space); - }); -} - -void MergeSortingStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void MergeSortingStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h deleted file mode 100644 index 947ced829c6..00000000000 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ /dev/null @@ -1,47 +0,0 @@ -#pragma once -#include -#include -#include -#include - -namespace DB -{ - -/// Sorts stream of data. See MergeSortingTransform. -class MergeSortingStep : public ITransformingStep -{ -public: - explicit MergeSortingStep( - const DataStream & input_stream, - const SortDescription & description_, - size_t max_merged_block_size_, - UInt64 limit_, - size_t max_bytes_before_remerge_, - double remerge_lowered_memory_bytes_ratio_, - size_t max_bytes_before_external_sort_, - VolumePtr tmp_volume_, - size_t min_free_disk_space_); - - String getName() const override { return "MergeSorting"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription description; - size_t max_merged_block_size; - UInt64 limit; - - size_t max_bytes_before_remerge; - double remerge_lowered_memory_bytes_ratio; - size_t max_bytes_before_external_sort; - VolumePtr tmp_volume; - size_t min_free_disk_space; -}; - -} diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp deleted file mode 100644 index ed1f24db55b..00000000000 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ /dev/null @@ -1,85 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = true, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -MergingSortedStep::MergingSortedStep( - const DataStream & input_stream, - SortDescription sort_description_, - size_t max_block_size_, - UInt64 limit_) - : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) - , sort_description(std::move(sort_description_)) - , max_block_size(max_block_size_) - , limit(limit_) -{ - /// TODO: check input_stream is partially sorted (each port) by the same description. - output_stream->sort_description = sort_description; - output_stream->sort_mode = DataStream::SortMode::Stream; -} - -void MergingSortedStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void MergingSortedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - /// If there are several streams, then we merge them into one - if (pipeline.getNumStreams() > 1) - { - - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - sort_description, - max_block_size, limit); - - pipeline.addTransform(std::move(transform)); - } -} - -void MergingSortedStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(sort_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void MergingSortedStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h deleted file mode 100644 index e886de42ca8..00000000000 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include -#include -#include - -namespace DB -{ - -/// Merge streams of data into single sorted stream. -class MergingSortedStep : public ITransformingStep -{ -public: - explicit MergingSortedStep( - const DataStream & input_stream, - SortDescription sort_description_, - size_t max_block_size_, - UInt64 limit_ = 0); - - String getName() const override { return "MergingSorted"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription sort_description; - size_t max_block_size; - UInt64 limit; -}; - -} diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp deleted file mode 100644 index 420e7b583ca..00000000000 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = true, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -PartialSortingStep::PartialSortingStep( - const DataStream & input_stream, - SortDescription sort_description_, - UInt64 limit_, - SizeLimits size_limits_) - : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) - , sort_description(std::move(sort_description_)) - , limit(limit_) - , size_limits(size_limits_) -{ - output_stream->sort_description = sort_description; - output_stream->sort_mode = DataStream::SortMode::Chunk; -} - -void PartialSortingStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void PartialSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - return std::make_shared(header, sort_description, limit); - }); - - StreamLocalLimits limits; - limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 - limits.size_limits = size_limits; - - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - auto transform = std::make_shared(header, limits); - return transform; - }); -} - -void PartialSortingStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(sort_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void PartialSortingStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h deleted file mode 100644 index 9b7b8e8baa5..00000000000 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -/// Sort separate chunks of data. -class PartialSortingStep : public ITransformingStep -{ -public: - explicit PartialSortingStep( - const DataStream & input_stream, - SortDescription sort_description_, - UInt64 limit_, - SizeLimits size_limits_); - - String getName() const override { return "PartialSorting"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription sort_description; - UInt64 limit; - SizeLimits size_limits; -}; - -} diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp new file mode 100644 index 00000000000..32b314b1c50 --- /dev/null +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -0,0 +1,241 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits(size_t limit) +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = limit == 0, + } + }; +} + +SortingStep::SortingStep( + const DataStream & input_stream, + const SortDescription & description_, + size_t max_block_size_, + UInt64 limit_, + SizeLimits size_limits_, + size_t max_bytes_before_remerge_, + double remerge_lowered_memory_bytes_ratio_, + size_t max_bytes_before_external_sort_, + VolumePtr tmp_volume_, + size_t min_free_disk_space_) + : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) + , type(Type::Full) + , result_description(description_) + , max_block_size(max_block_size_) + , limit(limit_) + , size_limits(size_limits_) + , max_bytes_before_remerge(max_bytes_before_remerge_) + , remerge_lowered_memory_bytes_ratio(remerge_lowered_memory_bytes_ratio_) + , max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_volume(tmp_volume_) + , min_free_disk_space(min_free_disk_space_) +{ + /// TODO: check input_stream is partially sorted by the same description. + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; +} + +SortingStep::SortingStep( + const DataStream & input_stream_, + SortDescription prefix_description_, + SortDescription result_description_, + size_t max_block_size_, + UInt64 limit_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits(limit_)) + , type(Type::FinishSorting) + , prefix_description(std::move(prefix_description_)) + , result_description(std::move(result_description_)) + , max_block_size(max_block_size_) + , limit(limit_) +{ + /// TODO: check input_stream is sorted by prefix_description. + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; +} + +SortingStep::SortingStep( + const DataStream & input_stream, + SortDescription sort_description_, + size_t max_block_size_, + UInt64 limit_) + : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) + , type(Type::MergingSorted) + , result_description(std::move(sort_description_)) + , max_block_size(max_block_size_) + , limit(limit_) +{ + /// TODO: check input_stream is partially sorted (each port) by the same description. + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; +} + +void SortingStep::updateLimit(size_t limit_) +{ + if (limit_ && (limit == 0 || limit_ < limit)) + { + limit = limit_; + transform_traits.preserves_number_of_rows = false; + } +} + +void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + if (type == Type::FinishSorting) + { + bool need_finish_sorting = (prefix_description.size() < result_description.size()); + if (pipeline.getNumStreams() > 1) + { + UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + prefix_description, + max_block_size, + limit_for_merging); + + pipeline.addTransform(std::move(transform)); + } + + if (need_finish_sorting) + { + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + return std::make_shared(header, result_description, limit); + }); + + /// NOTE limits are not applied to the size of temporary sets in FinishSortingTransform + pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr + { + return std::make_shared( + header, prefix_description, result_description, max_block_size, limit); + }); + } + } + else if (type == Type::Full) + { + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + return std::make_shared(header, result_description, limit); + }); + + StreamLocalLimits limits; + limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 + limits.size_limits = size_limits; + + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + auto transform = std::make_shared(header, limits); + return transform; + }); + + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type == QueryPipelineBuilder::StreamType::Totals) + return nullptr; + + return std::make_shared( + header, result_description, max_block_size, limit, + max_bytes_before_remerge / pipeline.getNumStreams(), + remerge_lowered_memory_bytes_ratio, + max_bytes_before_external_sort, + tmp_volume, + min_free_disk_space); + }); + + /// If there are several streams, then we merge them into one + if (pipeline.getNumStreams() > 1) + { + + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + result_description, + max_block_size, limit); + + pipeline.addTransform(std::move(transform)); + } + } + else if (type == Type::MergingSorted) + { /// If there are several streams, then we merge them into one + if (pipeline.getNumStreams() > 1) + { + + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + result_description, + max_block_size, limit); + + pipeline.addTransform(std::move(transform)); + } + } +} + +void SortingStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + + if (!prefix_description.empty()) + { + settings.out << prefix << "Prefix sort description: "; + dumpSortDescription(prefix_description, input_streams.front().header, settings.out); + settings.out << '\n'; + + settings.out << prefix << "Result sort description: "; + dumpSortDescription(result_description, input_streams.front().header, settings.out); + settings.out << '\n'; + } + else + { + settings.out << prefix << "Sort description: "; + dumpSortDescription(result_description, input_streams.front().header, settings.out); + settings.out << '\n'; + } + + if (limit) + settings.out << prefix << "Limit " << limit << '\n'; +} + +void SortingStep::describeActions(JSONBuilder::JSONMap & map) const +{ + if (!prefix_description.empty()) + { + map.add("Prefix Sort Description", explainSortDescription(prefix_description, input_streams.front().header)); + map.add("Result Sort Description", explainSortDescription(result_description, input_streams.front().header)); + } + else + map.add("Sort Description", explainSortDescription(result_description, input_streams.front().header)); + + if (limit) + map.add("Limit", limit); +} + +} diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h new file mode 100644 index 00000000000..e34ee2842da --- /dev/null +++ b/src/Processors/QueryPlan/SortingStep.h @@ -0,0 +1,76 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +/// Sort data stream +class SortingStep : public ITransformingStep +{ +public: + /// Full + SortingStep( + const DataStream & input_stream, + const SortDescription & description_, + size_t max_block_size_, + UInt64 limit_, + SizeLimits size_limits_, + size_t max_bytes_before_remerge_, + double remerge_lowered_memory_bytes_ratio_, + size_t max_bytes_before_external_sort_, + VolumePtr tmp_volume_, + size_t min_free_disk_space_); + + /// FinishSorting + SortingStep( + const DataStream & input_stream_, + SortDescription prefix_description_, + SortDescription result_description_, + size_t max_block_size_, + UInt64 limit_); + + /// MergingSorted + SortingStep( + const DataStream & input_stream, + SortDescription sort_description_, + size_t max_block_size_, + UInt64 limit_ = 0); + + String getName() const override { return "Sorting"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + + /// Add limit or change it to lower value. + void updateLimit(size_t limit_); + +private: + + enum class Type + { + Full, + FinishSorting, + MergingSorted, + }; + + Type type; + + SortDescription prefix_description; + SortDescription result_description; + size_t max_block_size; + UInt64 limit; + SizeLimits size_limits; + + size_t max_bytes_before_remerge; + double remerge_lowered_memory_bytes_ratio; + size_t max_bytes_before_external_sort; + VolumePtr tmp_volume; + size_t min_free_disk_space; +}; + +} From b7a53df9de5b8ec3b49fa6a5ee897b12c4757835 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Oct 2021 14:53:42 +0000 Subject: [PATCH 812/950] Send columns description in clickhouse-local --- src/Client/LocalConnection.cpp | 37 +++++++++++++++++-- src/Client/LocalConnection.h | 2 + ...khouse_local_columns_description.reference | 1 + ...00_clickhouse_local_columns_description.sh | 7 ++++ 4 files changed, 44 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03000_clickhouse_local_columns_description.reference create mode 100755 tests/queries/0_stateless/03000_clickhouse_local_columns_description.sh diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 4455ba3b9ad..20f28bb5337 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -5,7 +5,7 @@ #include #include #include -#include "Core/Protocol.h" +#include namespace DB @@ -105,6 +105,16 @@ void LocalConnection::sendQuery( state->pushing_executor->start(); state->block = state->pushing_executor->getHeader(); } + + const auto & table_id = query_context->getInsertionTable(); + if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) + { + if (!table_id.empty()) + { + auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, query_context); + state->columns_description = storage_ptr->getInMemoryMetadataPtr()->getColumns(); + } + } } else if (state->io.pipeline.pulling()) { @@ -117,7 +127,9 @@ void LocalConnection::sendQuery( executor.execute(); } - if (state->block) + if (state->columns_description) + next_packet_type = Protocol::Server::TableColumns; + else if (state->block) next_packet_type = Protocol::Server::Data; } catch (const Exception & e) @@ -337,21 +349,41 @@ Packet LocalConnection::receivePacket() packet.block = std::move(state->block.value()); state->block.reset(); } + next_packet_type.reset(); + break; + } + case Protocol::Server::TableColumns: + { + if (state->columns_description) + { + /// Send external table name (empty name is the main table) + /// (see TCPHandler::sendTableColumns) + packet.multistring_message = {"", state->columns_description->toString()}; + } + + if (state->block) + { + next_packet_type = Protocol::Server::Data; + } + break; } case Protocol::Server::Exception: { packet.exception = std::make_unique(*state->exception); + next_packet_type.reset(); break; } case Protocol::Server::Progress: { packet.progress = std::move(state->progress); state->progress.reset(); + next_packet_type.reset(); break; } case Protocol::Server::EndOfStream: { + next_packet_type.reset(); break; } default: @@ -359,7 +391,6 @@ Packet LocalConnection::receivePacket() "Unknown packet {} for {}", toString(packet.type), getDescription()); } - next_packet_type.reset(); return packet; } diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index b596360db64..1cc23defa6e 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -33,6 +34,7 @@ struct LocalQueryState /// Current block to be sent next. std::optional block; + std::optional columns_description; /// Is request cancelled bool is_cancelled = false; diff --git a/tests/queries/0_stateless/03000_clickhouse_local_columns_description.reference b/tests/queries/0_stateless/03000_clickhouse_local_columns_description.reference new file mode 100644 index 00000000000..0e291f3ac0d --- /dev/null +++ b/tests/queries/0_stateless/03000_clickhouse_local_columns_description.reference @@ -0,0 +1 @@ +1 42 diff --git a/tests/queries/0_stateless/03000_clickhouse_local_columns_description.sh b/tests/queries/0_stateless/03000_clickhouse_local_columns_description.sh new file mode 100755 index 00000000000..f88a8de80f5 --- /dev/null +++ b/tests/queries/0_stateless/03000_clickhouse_local_columns_description.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} --query "create table t (n int, m int default 42) engine=Memory;insert into t values (1, NULL);select * from t" From 1d9cfc04ef8873cbd16a1fe040e2e7f13640777d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 18:27:07 +0300 Subject: [PATCH 813/950] improve performance of aggregation in order of primary key --- src/Processors/Chunk.h | 1 + .../FinishAggregatingInOrderAlgorithm.cpp | 60 ++++++------------- .../FinishAggregatingInOrderAlgorithm.h | 15 ++--- .../FinishAggregatingInOrderTransform.h | 8 +-- src/Processors/Merges/IMergingTransform.cpp | 2 +- src/Processors/Merges/IMergingTransform.h | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 30 +++++----- .../AggregatingInOrderTransform.cpp | 2 +- .../Transforms/AggregatingInOrderTransform.h | 6 +- ...gingAggregatedMemoryEfficientTransform.cpp | 7 --- ...ergingAggregatedMemoryEfficientTransform.h | 7 +++ ...1_mergetree_read_in_order_spread.reference | 31 +++++----- 12 files changed, 70 insertions(+), 101 deletions(-) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index bb8266d6e27..011d719d094 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -82,6 +82,7 @@ public: MutableColumns cloneEmptyColumns() const; const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } + bool hasChunkInfo() const { return chunk_info != nullptr; } void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } UInt64 getNumRows() const { return num_rows; } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index 99e3adf893c..f54331d5550 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -25,14 +26,12 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_, - size_t merge_threads_) + size_t max_block_size_) : header(header_) , num_inputs(num_inputs_) , params(params_) , description(std::move(description_)) , max_block_size(max_block_size_) - , pool(merge_threads_) { /// Replace column names in description to positions. for (auto & column_description : description) @@ -60,12 +59,6 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() { - if (finished) - { - auto res = popResult(); - return Status(std::move(res), results.empty()); - } - if (!inputs_to_update.empty()) { Status status(inputs_to_update.back()); @@ -89,13 +82,7 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() } if (!best_input) - { - aggregate(); - pool.wait(); - finished = true; - auto res = popResult(); - return Status(std::move(res), results.empty()); - } + return Status(prepareToMerge(), true); /// Chunk at best_input will be aggregated entirely. auto & best_state = states[*best_input]; @@ -126,34 +113,20 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() /// Do not merge blocks, if there are too few rows. if (accumulated_rows >= max_block_size) - aggregate(); + status.chunk = prepareToMerge(); - status.chunk = popResult(); return status; } -Chunk FinishAggregatingInOrderAlgorithm::popResult() -{ - std::lock_guard lock(results_mutex); - - if (results.empty()) - return {}; - - auto res = std::move(results.back()); - results.pop_back(); - return res; -} - -void FinishAggregatingInOrderAlgorithm::aggregate() +Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() { accumulated_rows = 0; - pool.scheduleOrThrowOnError([this, blocks_list = std::move(blocks)]() mutable - { - auto aggregated = params->aggregator.mergeBlocks(blocks_list, false); + auto info = std::make_shared(); + info->chunks = std::make_unique(std::move(chunks)); - std::lock_guard lock(results_mutex); - results.emplace_back(aggregated.getColumns(), aggregated.rows()); - }); + Chunk chunk; + chunk.setChunkInfo(std::move(info)); + return chunk; } void FinishAggregatingInOrderAlgorithm::addToAggregation() @@ -164,22 +137,25 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() if (!state.isValid() || state.current_row == state.to_row) continue; - if (state.to_row - state.current_row == state.num_rows) + size_t current_rows = state.to_row - state.current_row; + if (current_rows == state.num_rows) { - blocks.emplace_back(header.cloneWithColumns(state.all_columns)); + chunks.emplace_back(state.all_columns, current_rows); } else { Columns new_columns; new_columns.reserve(state.all_columns.size()); for (const auto & column : state.all_columns) - new_columns.emplace_back(column->cut(state.current_row, state.to_row - state.current_row)); + new_columns.emplace_back(column->cut(state.current_row, current_rows)); - blocks.emplace_back(header.cloneWithColumns(new_columns)); + chunks.emplace_back(std::move(new_columns), current_rows); } + chunks.back().setChunkInfo(std::make_shared()); + states[i].current_row = states[i].to_row; - accumulated_rows += blocks.back().rows(); + accumulated_rows += current_rows; if (!states[i].isValid()) inputs_to_update.push_back(i); diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 8b83e4e215b..2591e11a211 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -39,17 +39,15 @@ public: size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_, - size_t merge_threads_); + size_t max_block_size_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; Status merge() override; private: - void aggregate(); + Chunk prepareToMerge(); void addToAggregation(); - Chunk popResult(); struct State { @@ -72,17 +70,14 @@ private: AggregatingTransformParamsPtr params; SortDescription description; size_t max_block_size; - ThreadPool pool; - - std::mutex results_mutex; - std::vector results; Inputs current_inputs; + std::vector states; std::vector inputs_to_update; - BlocksList blocks; + + std::vector chunks; size_t accumulated_rows = 0; - bool finished = false; }; } diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 3a388a36727..5605cea8e39 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -17,16 +17,14 @@ public: size_t num_inputs, AggregatingTransformParamsPtr params, SortDescription description, - size_t max_block_size, - size_t merge_threads) + size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, {}, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, header, num_inputs, params, std::move(description), - max_block_size, - merge_threads) + max_block_size) { } diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index cba78390c97..85416673d4f 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -132,7 +132,7 @@ IProcessor::Status IMergingTransformBase::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if (state.output_chunk && !is_port_full) + if ((state.output_chunk || state.output_chunk.hasChunkInfo()) && !is_port_full) output.push(std::move(state.output_chunk)); if (!is_initialized) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 8b0a44ae025..1acd4916260 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -107,7 +107,7 @@ public: IMergingAlgorithm::Status status = algorithm.merge(); - if (status.chunk && status.chunk.hasRows()) + if ((status.chunk && status.chunk.hasRows()) || status.chunk.hasChunkInfo()) { // std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl; state.output_chunk = std::move(status.chunk); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 4942757cbd8..0a362c4750a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -87,24 +88,21 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B aggregating_in_order = collector.detachProcessors(0); - for (auto & column_description : group_by_sort_description) - { - if (!column_description.column_name.empty()) - { - column_description.column_number = pipeline.getHeader().getPositionByName(column_description.column_name); - column_description.column_name.clear(); - } - } - auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), transform_params, group_by_sort_description, - max_block_size, - merge_threads); + max_block_size); pipeline.addTransform(std::move(transform)); + pipeline.resize(merge_threads); + + pipeline.addSimpleTransform([&](const Block &) + { + return std::make_shared(transform_params); + }); + aggregating_sorted = collector.detachProcessors(1); } else @@ -114,14 +112,14 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return std::make_shared(header, transform_params, group_by_sort_description, max_block_size); }); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, transform_params); + }); + aggregating_in_order = collector.detachProcessors(0); } - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, transform_params); - }); - finalizing = collector.detachProcessors(2); return; } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 33bc5280422..d6c5491f45c 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -133,7 +133,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) variants.without_key = nullptr; - /// Arenas cannot be destroyed here, since later, in FinalizingSimpleTransform + /// Arenas cannot be destroyed here, since later, in FinalizeAggregatedTransform /// there will be finalizeChunk(), but even after /// finalizeChunk() we cannot destroy arena, since some memory /// from Arena still in use, so we attach it to the Chunk to diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 10793e885ce..e0cb9b80967 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -64,10 +64,10 @@ private: }; -class FinalizingSimpleTransform : public ISimpleTransform +class FinalizeAggregatedTransform : public ISimpleTransform { public: - FinalizingSimpleTransform(Block header, AggregatingTransformParamsPtr params_) + FinalizeAggregatedTransform(Block header, AggregatingTransformParamsPtr params_) : ISimpleTransform({std::move(header)}, {params_->getHeader()}, true) , params(params_) {} @@ -82,7 +82,7 @@ public: } } - String getName() const override { return "FinalizingSimpleTransform"; } + String getName() const override { return "FinalizeAggregatedTransform"; } private: AggregatingTransformParamsPtr params; diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index df2ea4b03f0..0eec02809e4 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -12,13 +12,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -struct ChunksToMerge : public ChunkInfo -{ - std::unique_ptr chunks; - Int32 bucket_num = -1; - bool is_overflows = false; -}; - GroupingAggregatedTransform::GroupingAggregatedTransform( const Block & header_, size_t num_inputs_, AggregatingTransformParamsPtr params_) : IProcessor(InputPorts(num_inputs_, header_), { Block() }) diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 4367f6fec32..b4a62f8a13e 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -137,6 +137,13 @@ private: void addChunk(Chunk chunk, size_t from_input); }; +struct ChunksToMerge : public ChunkInfo +{ + std::unique_ptr chunks; + Int32 bucket_num = -1; + bool is_overflows = false; +}; + class Pipe; /// Adds processors to pipe which performs memory efficient merging of partially aggregated data from several sources. diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference index cdc595a3c57..b6b8b04907c 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -1,17 +1,18 @@ (Expression) -ExpressionTransform +ExpressionTransform × 3 (Aggregating) - FinalizingSimpleTransform - FinishAggregatingInOrderTransform 3 → 1 - AggregatingInOrderTransform × 3 - (Expression) - ExpressionTransform × 3 - (SettingQuotaAndLimits) - (ReadFromMergeTree) - ExpressionTransform × 4 - MergeTreeInOrder 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeInOrder × 2 0 → 1 - ExpressionTransform - MergeTreeInOrder 0 → 1 + MergingAggregatedBucketTransform × 3 + Resize 1 → 3 + FinishAggregatingInOrderTransform 3 → 1 + AggregatingInOrderTransform × 3 + (Expression) + ExpressionTransform × 3 + (SettingQuotaAndLimits) + (ReadFromMergeTree) + ExpressionTransform × 4 + MergeTreeInOrder 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeInOrder × 2 0 → 1 + ExpressionTransform + MergeTreeInOrder 0 → 1 From ab0d2504e57682711ae22ec83d3df642b1f0455a Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 15:44:54 +0000 Subject: [PATCH 814/950] fix --- src/Storages/FileLog/StorageFileLog.cpp | 15 +++++++++------ src/Storages/FileLog/StorageFileLog.h | 6 ++++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 92c30a86b49..7f4522841de 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -327,6 +327,7 @@ Pipe StorageFileLog::read( throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } + std::lock_guard lock(file_infos_mutex); updateFileInfos(); /// No files to parse @@ -578,16 +579,17 @@ size_t StorageFileLog::getTableDependentCount() const void StorageFileLog::threadFunc() { + bool reschedule = false; try { - updateFileInfos(); - + std::lock_guard lock(file_infos_mutex); auto table_id = getStorageID(); auto dependencies_count = getTableDependentCount(); if (dependencies_count) { + has_dependent_mv = true; auto start_time = std::chrono::steady_clock::now(); // Keep streaming as long as there are attached views and streaming is not cancelled @@ -616,6 +618,7 @@ void StorageFileLog::threadFunc() if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + reschedule = true; break; } } @@ -631,7 +634,7 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - if (!getTableDependentCount()) + if (!getTableDependentCount() || reschedule) task->holder->scheduleAfter(milliseconds_to_wait); else { @@ -654,9 +657,9 @@ bool StorageFileLog::streamToViews() { if (running_streams) { - throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); + LOG_INFO(log, "Another select query is running on this table, need to wait it finish."); + return true; } - has_dependent_mv = true; Stopwatch watch; auto table_id = getStorageID(); @@ -670,7 +673,7 @@ bool StorageFileLog::streamToViews() if (max_streams_number == 0) { LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); - return false; + return updateFileInfos(); } // Create an INSERT query for streaming data diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f0aac1a7985..e1d95ae736f 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -166,12 +166,14 @@ private: /// later select should forbid to execute. std::atomic running_streams = 0; - std::atomic has_dependent_mv = false; - std::mutex mutex; bool has_new_events = false; std::condition_variable cv; + bool has_dependent_mv = false; + + std::mutex file_infos_mutex; + struct TaskContext { BackgroundSchedulePool::TaskHolder holder; From 7962eddf60c94da5e4d0b25a2853c516bff809b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 19:18:09 +0300 Subject: [PATCH 815/950] Review fixes. --- src/Processors/Merges/IMergingTransform.cpp | 2 +- src/Processors/Merges/IMergingTransform.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index 2e0e21b7fe0..52acf36a4d7 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -15,7 +15,7 @@ IMergingTransformBase::IMergingTransformBase( const Block & input_header, const Block & output_header, bool have_all_inputs_, - size_t limit_hint_) + UInt64 limit_hint_) : IProcessor(InputPorts(num_inputs, input_header), {output_header}) , have_all_inputs(have_all_inputs_) , limit_hint(limit_hint_) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 155b705450b..4da49b8155c 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -17,7 +17,7 @@ public: const Block & input_header, const Block & output_header, bool have_all_inputs_, - size_t limit_hint_); + UInt64 limit_hint_); OutputPort & getOutputPort() { return outputs.front(); } @@ -67,7 +67,7 @@ private: std::vector input_states; std::atomic have_all_inputs; bool is_initialized = false; - size_t limit_hint = 0; + UInt64 limit_hint = 0; IProcessor::Status prepareInitializeInputs(); }; @@ -83,7 +83,7 @@ public: const Block & input_header, const Block & output_header, bool have_all_inputs_, - size_t limit_hint_, + UInt64 limit_hint_, Args && ... args) : IMergingTransformBase(num_inputs, input_header, output_header, have_all_inputs_, limit_hint_) , algorithm(std::forward(args) ...) From ef3597f0543b8ed1b02285934654a6bcc4daf635 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 19:21:49 +0300 Subject: [PATCH 816/950] Fix build. --- .../Optimizations/filterPushDown.cpp | 10 ++----- .../QueryPlan/Optimizations/limitPushDown.cpp | 28 +++---------------- 2 files changed, 6 insertions(+), 32 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 687946659b6..e81cec723a1 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -7,10 +7,7 @@ #include #include #include -#include -#include -#include -#include +#include #include #include #include @@ -237,10 +234,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes // { // } - if (typeid_cast(child.get()) - || typeid_cast(child.get()) - || typeid_cast(child.get()) - || typeid_cast(child.get())) + if (typeid_cast(child.get())) { Names allowed_inputs = child->getOutputStream().header.getNames(); if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) diff --git a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp index 01af6a2bbde..eb65f49103b 100644 --- a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp @@ -2,10 +2,7 @@ #include #include #include -#include -#include -#include -#include +#include #include namespace DB::QueryPlanOptimizations @@ -21,32 +18,15 @@ static bool tryUpdateLimitForSortingSteps(QueryPlan::Node * node, size_t limit) QueryPlan::Node * child = nullptr; bool updated = false; - if (auto * merging_sorted = typeid_cast(step.get())) + if (auto * sorting = typeid_cast(step.get())) { /// TODO: remove LimitStep here. - merging_sorted->updateLimit(limit); + sorting->updateLimit(limit); updated = true; child = node->children.front(); } - else if (auto * finish_sorting = typeid_cast(step.get())) - { - /// TODO: remove LimitStep here. - finish_sorting->updateLimit(limit); - updated = true; - } - else if (auto * merge_sorting = typeid_cast(step.get())) - { - merge_sorting->updateLimit(limit); - updated = true; - child = node->children.front(); - } - else if (auto * partial_sorting = typeid_cast(step.get())) - { - partial_sorting->updateLimit(limit); - updated = true; - } - /// We often have chain PartialSorting -> MergeSorting -> MergingSorted + /// In case we have several sorting steps. /// Try update limit for them also if possible. if (child) tryUpdateLimitForSortingSteps(child, limit); From 3e742524b45204a812a9fe76d3ad5afbd681b9fd Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 19:15:46 +0300 Subject: [PATCH 817/950] update comments --- .../FinishAggregatingInOrderAlgorithm.h | 17 ++++++++++------- src/Processors/QueryPlan/AggregatingStep.cpp | 9 +++++++++ 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 2591e11a211..209bb9e4045 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -16,20 +16,23 @@ using AggregatingTransformParamsPtr = std::shared_ptr 1) { + /** The pipeline is the following: + * + * --> AggregatingInOrder --> MergingAggregatedBucket + * --> AggregatingInOrder --> FinishAggregatingInOrder --> ResizeProcessor --> MergingAggregatedBucket + * --> AggregatingInOrder --> MergingAggregatedBucket + */ + auto many_data = std::make_shared(pipeline.getNumStreams()); size_t counter = 0; pipeline.addSimpleTransform([&](const Block & header) @@ -96,6 +103,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B max_block_size); pipeline.addTransform(std::move(transform)); + + /// Do merge of aggregated data in parallel. pipeline.resize(merge_threads); pipeline.addSimpleTransform([&](const Block &) From 34d46245c3987a726c37bd3ed13c54f06ae99c83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 20:53:30 +0300 Subject: [PATCH 818/950] Try fix some tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../QueryPlan/FinishSortingStep.cpp | 115 ------------------ src/Processors/QueryPlan/FinishSortingStep.h | 36 ------ 3 files changed, 1 insertion(+), 151 deletions(-) delete mode 100644 src/Processors/QueryPlan/FinishSortingStep.cpp delete mode 100644 src/Processors/QueryPlan/FinishSortingStep.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c6ef18d7660..494497fce9a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2329,6 +2329,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo settings.min_free_disk_space_for_temporary_data); sorting_step->setStepDescription("Sorting for ORDER BY"); + query_plan.addStep(std::move(sorting_step)); } diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp deleted file mode 100644 index 9002c804e7b..00000000000 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ /dev/null @@ -1,115 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = true, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -FinishSortingStep::FinishSortingStep( - const DataStream & input_stream_, - SortDescription prefix_description_, - SortDescription result_description_, - size_t max_block_size_, - UInt64 limit_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits(limit_)) - , prefix_description(std::move(prefix_description_)) - , result_description(std::move(result_description_)) - , max_block_size(max_block_size_) - , limit(limit_) -{ - /// TODO: check input_stream is sorted by prefix_description. - output_stream->sort_description = result_description; - output_stream->sort_mode = DataStream::SortMode::Stream; -} - -void FinishSortingStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void FinishSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - bool need_finish_sorting = (prefix_description.size() < result_description.size()); - if (pipeline.getNumStreams() > 1) - { - UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - prefix_description, - max_block_size, - limit_for_merging); - - pipeline.addTransform(std::move(transform)); - } - - if (need_finish_sorting) - { - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - return std::make_shared(header, result_description, limit); - }); - - /// NOTE limits are not applied to the size of temporary sets in FinishSortingTransform - pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr - { - return std::make_shared( - header, prefix_description, result_description, max_block_size, limit); - }); - } -} - -void FinishSortingStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - - settings.out << prefix << "Prefix sort description: "; - dumpSortDescription(prefix_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - settings.out << prefix << "Result sort description: "; - dumpSortDescription(result_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void FinishSortingStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Prefix Sort Description", explainSortDescription(prefix_description, input_streams.front().header)); - map.add("Result Sort Description", explainSortDescription(result_description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/FinishSortingStep.h b/src/Processors/QueryPlan/FinishSortingStep.h deleted file mode 100644 index fd56c4353e7..00000000000 --- a/src/Processors/QueryPlan/FinishSortingStep.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -/// Finish sorting of pre-sorted data. See FinishSortingTransform. -class FinishSortingStep : public ITransformingStep -{ -public: - FinishSortingStep( - const DataStream & input_stream_, - SortDescription prefix_description_, - SortDescription result_description_, - size_t max_block_size_, - UInt64 limit_); - - String getName() const override { return "FinishSorting"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription prefix_description; - SortDescription result_description; - size_t max_block_size; - UInt64 limit; -}; - -} From 3ffca6e138c7e23faf8b23942ff8f0a22147e75f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 18 Oct 2021 21:04:40 +0300 Subject: [PATCH 819/950] ExecutablePool dictionary source fix borrow timeout milliseconds --- src/Dictionaries/ExecutablePoolDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 8d1122b1194..dce2ce94b93 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -100,7 +100,7 @@ Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy{ true /*terminate_in_destructor*/, configuration.command_termination_timeout }; auto shell_command = ShellCommand::execute(config); return shell_command; - }, configuration.max_command_execution_time * 1000); + }, configuration.max_command_execution_time * 10000); if (!result) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, From 4e3910e564efcfd308fb3bccb271a7f36aa3386b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:11:54 +0300 Subject: [PATCH 820/950] Add a test for adjusting last granula with max_read_buffer_size=0 --- ...ast_granula_adjust_LOGICAL_ERROR.reference | 8 ++++++++ ...2_last_granula_adjust_LOGICAL_ERROR.sql.j2 | 19 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.reference create mode 100644 tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.sql.j2 diff --git a/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.reference b/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.reference new file mode 100644 index 00000000000..d7d3ee8f362 --- /dev/null +++ b/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.reference @@ -0,0 +1,8 @@ +1 +1 +10 +10 +100 +100 +10000 +10000 diff --git a/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.sql.j2 b/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.sql.j2 new file mode 100644 index 00000000000..465aa22beb3 --- /dev/null +++ b/tests/queries/0_stateless/02052_last_granula_adjust_LOGICAL_ERROR.sql.j2 @@ -0,0 +1,19 @@ +-- Tags: long + +{% for rows_in_table in [1, 10, 100, 10000] %} +{% for wide in [0, 100000000] %} +drop table if exists data_02052_{{ rows_in_table }}_wide{{ wide }}; +create table data_02052_{{ rows_in_table }}_wide{{ wide }} (key Int, value String) +engine=MergeTree() +order by key +settings + min_bytes_for_wide_part={{ wide }} +as select number, repeat(toString(number), 5) from numbers({{ rows_in_table }}); + +-- avoid any optimizations with ignore(*) +select count(ignore(*)) from data_02052_{{ rows_in_table }}_wide{{ wide }} settings max_read_buffer_size=1, max_threads=1; +select count(ignore(*)) from data_02052_{{ rows_in_table }}_wide{{ wide }} settings max_read_buffer_size=0, max_threads=1; -- { serverError CANNOT_READ_ALL_DATA } + +drop table data_02052_{{ rows_in_table }}_wide{{ wide }}; +{% endfor %} +{% endfor %} From 41e6df0f4b410a83b386d50dd11c4339334bddbf Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Oct 2021 00:27:28 +0300 Subject: [PATCH 821/950] Remove unused offset_columns from MergeTreeReaderWide::readRows() --- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 206469da7be..29cc45a5c60 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -69,10 +69,6 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si size_t num_columns = columns.size(); checkNumberOfColumns(num_columns); - /// Pointers to offset columns that are common to the nested data structure columns. - /// If append is true, then the value will be equal to nullptr and will be used only to - /// check that the offsets column has been already read. - OffsetColumns offset_columns; std::unordered_map caches; std::unordered_set prefetched_streams; From cd4b33c8c9066f58594e21a605ce27d093026127 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:11:54 +0300 Subject: [PATCH 822/950] Verify that all rows was read in MergeTreeReaderCompact v0: Use fmt-like style exception in MergeTreeReaderCompact v2: Update the check --- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 9b879283c10..15c5795ee7b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -160,9 +160,10 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, readData(column_from_part, column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; - if (read_rows_in_column < rows_to_read) - throw Exception("Cannot read all data in MergeTreeReaderCompact. Rows read: " + toString(read_rows_in_column) + - ". Rows expected: " + toString(rows_to_read) + ".", ErrorCodes::CANNOT_READ_ALL_DATA); + if (read_rows_in_column != rows_to_read) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.", + read_rows_in_column, rows_to_read); } catch (Exception & e) { From e576fd17bd806666886cd8da7a9466d4268b88e7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Oct 2021 21:11:54 +0300 Subject: [PATCH 823/950] Do not allow reading to empty buffer in MergeTreeReaderStream Changelog: - Verify that all rows was read in MergeTreeReaderWide - Ignore some exceptions for Wide parts - Take max_rows_to_read into account - Verify that requested rows is not zero (otherwise it is too tricky) - Simply verify that buffer is not empty --- src/Storages/MergeTree/MergeTreeReaderStream.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index fc57b48e86d..f225ecae8fa 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; + extern const int CANNOT_READ_ALL_DATA; } @@ -76,6 +77,10 @@ MergeTreeReaderStream::MergeTreeReaderStream( if (max_mark_range_bytes != 0) read_settings = read_settings.adjustBufferSize(max_mark_range_bytes); + //// Empty buffer does not makes progress. + if (!read_settings.local_fs_buffer_size || !read_settings.remote_fs_buffer_size) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read to empty buffer."); + /// Initialize the objects that shall be used to perform read operations. if (uncompressed_cache) { From 0d2ce2e711fd0a5b4aa7a1061b848f12f22ccaa6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 18 Oct 2021 21:56:04 +0300 Subject: [PATCH 824/950] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 700c804ea66..498de206bb9 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -166,5 +166,6 @@ toc_title: Adopters | Beeline | Telecom | Data Platform | — | — | [Blog post, July 2021](https://habr.com/en/company/beeline/blog/567508/) | | Ecommpay | Payment Processing | Logs | — | — | [Video, Nov 2019](https://www.youtube.com/watch?v=d3GdZTOWGLk) | | Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, Oct 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | +| Ok.ru | Social Network | — | — | 72 servers, 810 TB, 50bn rows/day, 1.5 TB/day | [SmartData conference, Oct 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From 1c64a656f48d3632aa96592b15022264f8f9e8f8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 18 Oct 2021 21:56:43 +0300 Subject: [PATCH 825/950] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 498de206bb9..73db0b3c952 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -166,6 +166,6 @@ toc_title: Adopters | Beeline | Telecom | Data Platform | — | — | [Blog post, July 2021](https://habr.com/en/company/beeline/blog/567508/) | | Ecommpay | Payment Processing | Logs | — | — | [Video, Nov 2019](https://www.youtube.com/watch?v=d3GdZTOWGLk) | | Omnicomm | Transportation Monitoring | — | — | — | [Facebook post, Oct 2021](https://www.facebook.com/OmnicommTeam/posts/2824479777774500) | -| Ok.ru | Social Network | — | — | 72 servers, 810 TB, 50bn rows/day, 1.5 TB/day | [SmartData conference, Oct 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | +| Ok.ru | Social Network | — | 72 servers | 810 TB compressed, 50bn rows/day, 1.5 TB/day | [SmartData conference, Oct 2021](https://assets.ctfassets.net/oxjq45e8ilak/4JPHkbJenLgZhBGGyyonFP/57472ec6987003ec4078d0941740703b/____________________ClickHouse_______________________.pdf) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From f7b76373ceecc024eb93f3cbf5198b0022acdcc9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 13 Oct 2021 23:52:39 +0300 Subject: [PATCH 826/950] Move ProfileEvents packet type from TCPHandler into ProfileEventsExt --- src/Interpreters/ProfileEventsExt.cpp | 5 +++++ src/Interpreters/ProfileEventsExt.h | 10 ++++++++++ src/Server/TCPHandler.cpp | 20 ++++---------------- 3 files changed, 19 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index 4386c294316..472efc109fb 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -11,6 +11,11 @@ namespace ProfileEvents { +std::shared_ptr TypeEnum = std::make_shared(DB::DataTypeEnum8::Values{ + { "increment", static_cast(INCREMENT)}, + { "gauge", static_cast(GAUGE)}, +}); + /// Put implementation here to avoid extra linking dependencies for clickhouse_common_io void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only) { diff --git a/src/Interpreters/ProfileEventsExt.h b/src/Interpreters/ProfileEventsExt.h index 699c997d904..8a92eadec79 100644 --- a/src/Interpreters/ProfileEventsExt.h +++ b/src/Interpreters/ProfileEventsExt.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include @@ -9,4 +10,13 @@ namespace ProfileEvents /// Dumps profile events to columns Map(String, UInt64) void dumpToMapColumn(const Counters::Snapshot & counters, DB::IColumn * column, bool nonzero_only = true); +/// This is for ProfileEvents packets. +enum Type : int8_t +{ + INCREMENT = 1, + GAUGE = 2, +}; + +extern std::shared_ptr TypeEnum; + } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 87cc76b1812..729cb33371a 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -831,12 +832,6 @@ namespace { using namespace ProfileEvents; - enum ProfileEventTypes : int8_t - { - INCREMENT = 1, - GAUGE = 2, - }; - constexpr size_t NAME_COLUMN_INDEX = 4; constexpr size_t VALUE_COLUMN_INDEX = 5; @@ -879,7 +874,7 @@ namespace columns[i++]->insertData(host_name.data(), host_name.size()); columns[i++]->insert(UInt64(snapshot.current_time)); columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEventTypes::INCREMENT); + columns[i++]->insert(ProfileEvents::Type::INCREMENT); } } @@ -893,7 +888,7 @@ namespace columns[i++]->insertData(host_name.data(), host_name.size()); columns[i++]->insert(UInt64(snapshot.current_time)); columns[i++]->insert(UInt64{snapshot.thread_id}); - columns[i++]->insert(ProfileEventTypes::GAUGE); + columns[i++]->insert(ProfileEvents::Type::GAUGE); columns[i++]->insertData(MemoryTracker::USAGE_EVENT_NAME, strlen(MemoryTracker::USAGE_EVENT_NAME)); columns[i++]->insert(snapshot.memory_usage); @@ -907,18 +902,11 @@ void TCPHandler::sendProfileEvents() if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS) return; - auto profile_event_type = std::make_shared( - DataTypeEnum8::Values - { - { "increment", static_cast(INCREMENT)}, - { "gauge", static_cast(GAUGE)}, - }); - NamesAndTypesList column_names_and_types = { { "host_name", std::make_shared() }, { "current_time", std::make_shared() }, { "thread_id", std::make_shared() }, - { "type", profile_event_type }, + { "type", ProfileEvents::TypeEnum }, { "name", std::make_shared() }, { "value", std::make_shared() }, }; From 424bf6fcf49ce377a26270f9872e0939912dac79 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 12 Oct 2021 21:03:54 +0300 Subject: [PATCH 827/950] client: add ability to print raw profile events This can be useful for debugging and for testing (since you will not need to obtain query_id and look at query_log). v2: - mark test as long - add option to docs - add type of profile event into logs too v3: - resolve conflicts - and fix onProfileEvents callback v4: - add --print-profile-events separate switch --- docs/en/interfaces/cli.md | 2 + src/Client/ClientBase.cpp | 88 ++++++++++++++----- src/Client/ClientBase.h | 11 +++ src/Client/InternalTextLogs.cpp | 68 +++++++++++++- src/Client/InternalTextLogs.h | 29 +++++- .../02050_client_profile_events.reference | 4 + .../02050_client_profile_events.sh | 15 ++++ 7 files changed, 188 insertions(+), 29 deletions(-) create mode 100644 tests/queries/0_stateless/02050_client_profile_events.reference create mode 100755 tests/queries/0_stateless/02050_client_profile_events.sh diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index c4305d229cf..eaf7a96ce42 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -128,6 +128,8 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--history_file` — Path to a file containing command history. - `--param_` — Value for a [query with parameters](#cli-queries-with-parameters). - `--hardware-utilization` — Print hardware utilization information in progress bar. +- `--print-profile-events` – Print `ProfileEvents` packets. +- `--profile-events-delay-ms` – Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet). Since version 20.5, `clickhouse-client` has automatic syntax highlighting (always enabled). diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 7bcff4f5ef7..54e679e4c0f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -266,7 +266,7 @@ void ClientBase::onLogData(Block & block) { initLogsOutputStream(); progress_indication.clearProgressOutput(); - logs_out_stream->write(block); + logs_out_stream->writeLogs(block); logs_out_stream->flush(); } @@ -668,39 +668,61 @@ void ClientBase::onEndOfStream() void ClientBase::onProfileEvents(Block & block) { const auto rows = block.rows(); - if (rows == 0 || !progress_indication.print_hardware_utilization) + if (rows == 0) return; - const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); - const auto & names = typeid_cast(*block.getByName("name").column); - const auto & host_names = typeid_cast(*block.getByName("host_name").column); - const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); - const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); - const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); - - HostToThreadTimesMap thread_times; - for (size_t i = 0; i < rows; ++i) + if (progress_indication.print_hardware_utilization) { - auto thread_id = array_thread_id[i]; - auto host_name = host_names.getDataAt(i).toString(); - if (thread_id != 0) - progress_indication.addThreadIdToList(host_name, thread_id); - auto event_name = names.getDataAt(i); - auto value = array_values[i]; - if (event_name == user_time_name) + const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); + const auto & names = typeid_cast(*block.getByName("name").column); + const auto & host_names = typeid_cast(*block.getByName("host_name").column); + const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); + + const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); + const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); + + HostToThreadTimesMap thread_times; + for (size_t i = 0; i < rows; ++i) { - thread_times[host_name][thread_id].user_ms = value; + auto thread_id = array_thread_id[i]; + auto host_name = host_names.getDataAt(i).toString(); + if (thread_id != 0) + progress_indication.addThreadIdToList(host_name, thread_id); + auto event_name = names.getDataAt(i); + auto value = array_values[i]; + if (event_name == user_time_name) + { + thread_times[host_name][thread_id].user_ms = value; + } + else if (event_name == system_time_name) + { + thread_times[host_name][thread_id].system_ms = value; + } + else if (event_name == MemoryTracker::USAGE_EVENT_NAME) + { + thread_times[host_name][thread_id].memory_usage = value; + } } - else if (event_name == system_time_name) + progress_indication.updateThreadEventData(thread_times); + } + + if (profile_events.print) + { + if (profile_events.watch.elapsedMilliseconds() >= profile_events.delay_ms) { - thread_times[host_name][thread_id].system_ms = value; + initLogsOutputStream(); + progress_indication.clearProgressOutput(); + logs_out_stream->writeProfileEvents(block); + logs_out_stream->flush(); + + profile_events.watch.restart(); + profile_events.last_block = {}; } - else if (event_name == MemoryTracker::USAGE_EVENT_NAME) + else { - thread_times[host_name][thread_id].memory_usage = value; + profile_events.last_block = block; } } - progress_indication.updateThreadEventData(thread_times); } @@ -1023,6 +1045,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin processed_rows = 0; written_first_block = false; progress_indication.resetProgress(); + profile_events.watch.restart(); { /// Temporarily apply query settings to context. @@ -1091,6 +1114,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin } } + /// Always print last block (if it was not printed already) + if (profile_events.last_block) + { + initLogsOutputStream(); + progress_indication.clearProgressOutput(); + logs_out_stream->writeProfileEvents(profile_events.last_block); + logs_out_stream->flush(); + } + if (is_interactive) { std::cout << std::endl << processed_rows << " rows in set. Elapsed: " << progress_indication.elapsedSeconds() << " sec. "; @@ -1561,6 +1593,8 @@ void ClientBase::init(int argc, char ** argv) ("ignore-error", "do not stop processing in multiquery mode") ("stacktrace", "print stack traces of exceptions") ("hardware-utilization", "print hardware utilization information in progress bar") + ("print-profile-events", po::value(&profile_events.print)->zero_tokens(), "Printing ProfileEvents packets") + ("profile-events-delay-ms", po::value()->default_value(profile_events.delay_ms), "Delay between printing `ProfileEvents` packets (-1 - print only totals, 0 - print every single packet)") ; addAndCheckOptions(options_description, options, common_arguments); @@ -1611,6 +1645,10 @@ void ClientBase::init(int argc, char ** argv) config().setBool("vertical", true); if (options.count("stacktrace")) config().setBool("stacktrace", true); + if (options.count("print-profile-events")) + config().setBool("print-profile-events", true); + if (options.count("profile-events-delay-ms")) + config().setInt("profile-events-delay-ms", options["profile-events-delay-ms"].as()); if (options.count("progress")) config().setBool("progress", true); if (options.count("echo")) @@ -1631,6 +1669,8 @@ void ClientBase::init(int argc, char ** argv) progress_indication.print_hardware_utilization = true; query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); + profile_events.print = options.count("print-profile-events"); + profile_events.delay_ms = options["profile-events-delay-ms"].as(); processOptions(options_description, options, external_tables_arguments); argsToConfig(common_arguments, config(), 100); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index fce706d7cf8..b31eff82b30 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -217,6 +218,16 @@ protected: QueryFuzzer fuzzer; int query_fuzzer_runs = 0; + struct + { + bool print = false; + /// UINT64_MAX -- print only last + UInt64 delay_ms = 0; + Stopwatch watch; + /// For printing only last (delay_ms == 0). + Block last_block; + } profile_events; + QueryProcessingStage::Enum query_processing_stage; }; diff --git a/src/Client/InternalTextLogs.cpp b/src/Client/InternalTextLogs.cpp index 65592fee670..430ba6daf0a 100644 --- a/src/Client/InternalTextLogs.cpp +++ b/src/Client/InternalTextLogs.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -13,7 +14,7 @@ namespace DB { -void InternalTextLogs::write(const Block & block) +void InternalTextLogs::writeLogs(const Block & block) { const auto & array_event_time = typeid_cast(*block.getByName("event_time").column).getData(); const auto & array_microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData(); @@ -97,4 +98,69 @@ void InternalTextLogs::write(const Block & block) } } +void InternalTextLogs::writeProfileEvents(const Block & block) +{ + const auto & column_host_name = typeid_cast(*block.getByName("host_name").column); + const auto & array_current_time = typeid_cast(*block.getByName("current_time").column).getData(); + const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); + const auto & array_type = typeid_cast(*block.getByName("type").column).getData(); + const auto & column_name = typeid_cast(*block.getByName("name").column); + const auto & array_value = typeid_cast(*block.getByName("value").column).getData(); + + for (size_t row_num = 0; row_num < block.rows(); ++row_num) + { + /// host_name + auto host_name = column_host_name.getDataAt(row_num); + if (host_name.size) + { + writeCString("[", wb); + if (color) + writeString(setColor(StringRefHash()(host_name)), wb); + writeString(host_name, wb); + if (color) + writeCString(resetColor(), wb); + writeCString("] ", wb); + } + + /// current_time + auto current_time = array_current_time[row_num]; + writeDateTimeText<'.', ':'>(current_time, wb); + + /// thread_id + UInt64 thread_id = array_thread_id[row_num]; + writeCString(" [ ", wb); + if (color) + writeString(setColor(intHash64(thread_id)), wb); + writeIntText(thread_id, wb); + if (color) + writeCString(resetColor(), wb); + writeCString(" ] ", wb); + + /// name + auto name = column_name.getDataAt(row_num); + if (color) + writeString(setColor(StringRefHash()(name)), wb); + DB::writeString(name, wb); + if (color) + writeCString(resetColor(), wb); + writeCString(": ", wb); + + /// value + UInt64 value = array_value[row_num]; + writeIntText(value, wb); + + //// type + Int8 type = array_type[row_num]; + writeCString(" (", wb); + if (color) + writeString(setColor(intHash64(type)), wb); + writeString(toString(ProfileEvents::TypeEnum->castToName(type)), wb); + if (color) + writeCString(resetColor(), wb); + writeCString(")", wb); + + writeChar('\n', wb); + } +} + } diff --git a/src/Client/InternalTextLogs.h b/src/Client/InternalTextLogs.h index a8b119b0f69..0690211fd24 100644 --- a/src/Client/InternalTextLogs.h +++ b/src/Client/InternalTextLogs.h @@ -6,16 +6,37 @@ namespace DB { -/// Prints internal server logs -/// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock() +/// Prints internal server logs or profile events with colored output (if requested). /// NOTE: IRowOutputFormat does not suite well for this case class InternalTextLogs { public: InternalTextLogs(WriteBuffer & buf_out, bool color_) : wb(buf_out), color(color_) {} - - void write(const Block & block); + /// Print internal server logs + /// + /// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock(): + /// - event_time + /// - event_time_microseconds + /// - host_name + /// - query_id + /// - thread_id + /// - priority + /// - source + /// - text + void writeLogs(const Block & block); + /// Print profile events. + /// + /// Block: + /// - host_name + /// - current_time + /// - thread_id + /// - type + /// - name + /// - value + /// + /// See also TCPHandler::sendProfileEvents() for block columns. + void writeProfileEvents(const Block & block); void flush() { diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference new file mode 100644 index 00000000000..00fc3b5d06a --- /dev/null +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -0,0 +1,4 @@ +0 +SelectedRows: 131010 (increment) +OK +OK diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh new file mode 100755 index 00000000000..5c3887cf5fb --- /dev/null +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash +# Tags: long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# do not print any ProfileEvents packets +$CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows' +# print only last +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5) format Null' |& grep -o 'SelectedRows: .*$' +# print everything +test "$($CLICKHOUSE_CLIENT --print-profile-events -q 'select * from numbers(1e9) format Null' |& grep -c 'SelectedRows')" -gt 1 && echo OK || echo FAIL +# print each 100 ms +test "$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=100 -q 'select * from numbers(1e9) format Null' |& grep -c 'SelectedRows')" -gt 1 && echo OK || echo FAIL From fb91b1d92b8309e8a925cce25f4e1adaf967a306 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Oct 2021 01:04:07 +0300 Subject: [PATCH 828/950] StorageExecutable fix small issues --- src/Storages/StorageExecutable.cpp | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 76be3353808..6a82fc88977 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -3,6 +3,8 @@ #include #include +#include + #include #include @@ -111,9 +113,16 @@ Pipe StorageExecutable::read( { auto user_scripts_path = context->getUserScriptsPath(); auto script_path = user_scripts_path + '/' + script_name; - if (!std::filesystem::exists(std::filesystem::path(script_path))) + + if (!pathStartsWith(script_path, user_scripts_path)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Executable file {} does not exists inside {}", + "Executable file {} must be inside user scripts folder {}", + script_name, + user_scripts_path); + + if (!std::filesystem::exists(std::filesystem::path(script_path))) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Executable file {} does not exists inside user scripts folder {}", script_name, user_scripts_path); @@ -139,9 +148,9 @@ Pipe StorageExecutable::read( bool result = process_pool->tryBorrowObject(process, [&config, this]() { config.terminate_in_destructor_strategy = ShellCommand::DestructorStrategy{ true /*terminate_in_destructor*/, settings.command_termination_timeout }; - auto shell_command = ShellCommand::execute(config); + auto shell_command = ShellCommand::executeDirect(config); return shell_command; - }, settings.max_command_execution_time * 1000); + }, settings.max_command_execution_time * 10000); if (!result) throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, From 254eecf2270548206b60d9fbeab8b7795e87b3a8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 11:11:54 +0300 Subject: [PATCH 829/950] Update some tests. --- ...monotonous_functions_in_order_by.reference | 14 ++-- .../01576_alias_column_rewrite.reference | 18 +++-- .../01591_window_functions.reference | 26 +++---- .../01655_plan_optimizations.reference | 12 ++-- .../0_stateless/01655_plan_optimizations.sh | 4 +- ...1951_distributed_push_down_limit.reference | 28 ++++---- ...istributed_group_by_sharding_key.reference | 68 ++++++++----------- 7 files changed, 72 insertions(+), 98 deletions(-) diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference index bf9bff06959..d8c9b88d8e8 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -6,12 +6,10 @@ ORDER BY timestamp ASC LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromMergeTree + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromMergeTree SELECT timestamp, key @@ -20,7 +18,7 @@ ORDER BY toDate(timestamp) ASC LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree @@ -34,7 +32,7 @@ ORDER BY LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index 2a824e62158..07d361cfa46 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -23,21 +23,19 @@ lambda optimize_read_in_order Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromMergeTree -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromMergeTree +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index f54c10ee8b9..07562557369 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -908,12 +908,10 @@ Expression ((Projection + Before ORDER BY)) Window (Window step for window \'\') Window (Window step for window \'PARTITION BY p\') Window (Window step for window \'PARTITION BY p ORDER BY o ASC\') - MergingSorted (Merge sorted streams for window \'PARTITION BY p ORDER BY o ASC\') - MergeSorting (Merge sorted blocks for window \'PARTITION BY p ORDER BY o ASC\') - PartialSorting (Sort each block for window \'PARTITION BY p ORDER BY o ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + Sorting (Sorting for window \'PARTITION BY p ORDER BY o ASC\') + Expression ((Before window functions + (Projection + Before ORDER BY))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) explain select count(*) over (order by o, number), count(*) over (order by number) @@ -923,16 +921,12 @@ from ; Expression ((Projection + Before ORDER BY)) Window (Window step for window \'ORDER BY o ASC, number ASC\') - MergingSorted (Merge sorted streams for window \'ORDER BY o ASC, number ASC\') - MergeSorting (Merge sorted blocks for window \'ORDER BY o ASC, number ASC\') - PartialSorting (Sort each block for window \'ORDER BY o ASC, number ASC\') - Window (Window step for window \'ORDER BY number ASC\') - MergingSorted (Merge sorted streams for window \'ORDER BY number ASC\') - MergeSorting (Merge sorted blocks for window \'ORDER BY number ASC\') - PartialSorting (Sort each block for window \'ORDER BY number ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + Sorting (Sorting for window \'ORDER BY o ASC, number ASC\') + Window (Window step for window \'ORDER BY number ASC\') + Sorting (Sorting for window \'ORDER BY number ASC\') + Expression ((Before window functions + (Projection + Before ORDER BY))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) -- A test case for the sort comparator found by fuzzer. SELECT max(number) OVER (ORDER BY number DESC NULLS FIRST), diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 22876207862..33a7ff44b74 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -5,11 +5,8 @@ FUNCTION sipHash64 Filter column: equals > sorting steps should know about limit Limit 10 -MergingSorted -Limit 10 -MergeSorting -Limit 10 -PartialSorting +Sorting +Sorting Limit 10 -- filter push down -- > filter should be pushed down after aggregating @@ -108,9 +105,8 @@ Filter column: notEquals(y, 2) 1 0 1 1 > filter is pushed down before sorting steps -MergingSorted -MergeSorting -PartialSorting +Sorting +Sorting Filter column: and(notEquals(x, 0), notEquals(y, 0)) 1 2 1 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index b835bae0e27..de3d3ac3eb6 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "select x + 1 from (select y + 2 as x from (select dummy + echo "> sipHash should be calculated after filtration" $CLICKHOUSE_CLIENT -q "explain actions = 1 select sum(x), sum(y) from (select sipHash64(number) as x, bitAnd(number, 1024) as y from numbers_mt(1000000000) limit 1000000000) where y = 0" | grep -o "FUNCTION sipHash64\|Filter column: equals" echo "> sorting steps should know about limit" -$CLICKHOUSE_CLIENT -q "explain actions = 1 select number from (select number from numbers(500000000) order by -number) limit 10" | grep -o "MergingSorted\|MergeSorting\|PartialSorting\|Limit 10" +$CLICKHOUSE_CLIENT -q "explain actions = 1 select number from (select number from numbers(500000000) order by -number) limit 10" | grep -o "Sorting\|Limit 10" echo "-- filter push down --" echo "> filter should be pushed down after aggregating" @@ -132,7 +132,7 @@ $CLICKHOUSE_CLIENT -q " select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" | - grep -o "MergingSorted\|MergeSorting\|PartialSorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" + grep -o "Sorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" $CLICKHOUSE_CLIENT -q " select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index ca3bbc4cb33..d0e7a9ef15b 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -2,31 +2,27 @@ explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union Limit (preliminary LIMIT (with OFFSET)) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index 10787068f43..b23631395ff 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -50,66 +50,58 @@ SettingQuotaAndLimits (Set limits and quota after reading from storage) explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized Expression (Projection) Distinct - MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + Sorting (Merge sorted streams for ORDER BY, without aggregation) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Distinct (Preliminary DISTINCT) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized Expression (Projection) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union Distinct - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Distinct (Preliminary DISTINCT) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized Expression (Projection) LimitBy Expression (Before LIMIT BY) - MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + Sorting (Merge sorted streams for ORDER BY, without aggregation) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union LimitBy Expression (Before LIMIT BY) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) - ReadFromRemote (Read from remote replica) -explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized -Expression (Projection) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - LimitBy - Expression (Before LIMIT BY) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Projection) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression (Before LIMIT BY) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) From e53335bc6fc061ce47a40b94d3b5a91ac042717f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Oct 2021 08:19:43 +0000 Subject: [PATCH 830/950] Better way --- programs/server/Server.cpp | 2 +- src/Databases/DatabaseOnDisk.cpp | 9 --------- src/Interpreters/Context.cpp | 11 ++++------- src/Interpreters/Context.h | 3 +-- src/Interpreters/InterpreterCreateQuery.cpp | 9 --------- src/Storages/MergeTree/MergeTreeData.cpp | 2 ++ 6 files changed, 8 insertions(+), 28 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 2b526608715..bbd9af1e97e 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -919,7 +919,7 @@ if (ThreadFuzzer::instance().isEffective()) /// Initialize background executors after we load default_profile config. /// This is needed to load proper values of background_pool_size etc. - global_context->initializeBackgroundExecutors(); + global_context->initializeBackgroundExecutorsIfNeeded(); if (settings.async_insert_threads) global_context->setAsynchronousInsertQueue(std::make_shared( diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a71d539e3c5..97e59f53f64 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -53,15 +53,6 @@ std::pair createTableFromAST( ast_create_query.attach = true; ast_create_query.database = database_name; - auto global_context = context->getGlobalContext(); - if (global_context - && global_context->getApplicationType() == Context::ApplicationType::LOCAL - && !global_context->isBackgroundExecutorsInitialized() - && ast_create_query.storage && endsWith(ast_create_query.storage->engine->name, "MergeTree")) - { - global_context->initializeBackgroundExecutors(); - } - if (ast_create_query.as_table_function) { const auto & factory = TableFunctionFactory::instance(); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1602e6a6a31..bbad7e782ed 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2895,14 +2895,11 @@ void Context::setAsynchronousInsertQueue(const std::shared_ptrasync_insert_queue = ptr; } -bool Context::isBackgroundExecutorsInitialized() const +void Context::initializeBackgroundExecutorsIfNeeded() { - return is_background_executors_initialized; -} - -void Context::initializeBackgroundExecutors() -{ - assert(!is_background_executors_initialized); + auto lock = getLock(); + if (is_background_executors_initialized) + return; const size_t max_merges_and_mutations = getSettingsRef().background_pool_size * getSettingsRef().background_merges_mutations_concurrency_ratio; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 93be367e46d..b20274c2cb8 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -862,8 +862,7 @@ public: void setReadTaskCallback(ReadTaskCallback && callback); /// Background executors related methods - void initializeBackgroundExecutors(); - bool isBackgroundExecutorsInitialized() const; + void initializeBackgroundExecutorsIfNeeded(); MergeMutateBackgroundExecutorPtr getMergeMutateExecutor() const; OrdinaryBackgroundExecutorPtr getMovesExecutor() const; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 5b993bce724..6d38c55bd62 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -833,15 +833,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = getContext()->getCurrentDatabase(); auto database_name = create.database.empty() ? current_database : create.database; - auto global_context = getContext()->getGlobalContext(); - if (global_context - && global_context->getApplicationType() == Context::ApplicationType::LOCAL - && !global_context->isBackgroundExecutorsInitialized() - && create.storage && endsWith(create.storage->engine->name, "MergeTree")) - { - global_context->initializeBackgroundExecutors(); - } - // If this is a stub ATTACH query, read the query definition from the database if (create.attach && !create.storage && !create.columns_list) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 10fa18186ee..8b03c1e614d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -205,6 +205,8 @@ MergeTreeData::MergeTreeData( , background_operations_assignee(*this, BackgroundJobsAssignee::Type::DataProcessing, getContext()) , background_moves_assignee(*this, BackgroundJobsAssignee::Type::Moving, getContext()) { + context_->getGlobalContext()->initializeBackgroundExecutorsIfNeeded(); + const auto settings = getSettings(); allow_nullable_key = attach || settings->allow_nullable_key; From eb0ce68f10e860bfc864bcca7d6bcdeca2072ab4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Oct 2021 11:27:48 +0300 Subject: [PATCH 831/950] Update 02051_symlinks_to_user_files.sh --- tests/queries/0_stateless/02051_symlinks_to_user_files.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh index fe3073f9ff2..dfdc71e0f0b 100755 --- a/tests/queries/0_stateless/02051_symlinks_to_user_files.sh +++ b/tests/queries/0_stateless/02051_symlinks_to_user_files.sh @@ -19,7 +19,7 @@ file_path=$CUR_DIR/${FILE} touch ${file_path} ln -s ${file_path} ${symlink_path} -chmod +w ${symlink_path} +chmod ugo+w ${symlink_path} function cleanup() { From 40677bffa51f7ad8c905e47f81b32c80c16e5dd1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 12:28:49 +0300 Subject: [PATCH 832/950] Fix PVS warning. --- src/Processors/QueryPlan/SortingStep.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index e34ee2842da..8e253e71f44 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -66,11 +66,11 @@ private: UInt64 limit; SizeLimits size_limits; - size_t max_bytes_before_remerge; - double remerge_lowered_memory_bytes_ratio; - size_t max_bytes_before_external_sort; + size_t max_bytes_before_remerge = 0; + double remerge_lowered_memory_bytes_ratio = 0; + size_t max_bytes_before_external_sort = 0; VolumePtr tmp_volume; - size_t min_free_disk_space; + size_t min_free_disk_space = 0; }; } From 3dfbc80f0b0a4aec7649cf2678d4257a19b10b1a Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Oct 2021 12:38:11 +0300 Subject: [PATCH 833/950] Add cases to test replaceRegexpAll_bug --- .../02100_replaceRegexpAll_bug.reference | 14 +++++++++++--- .../0_stateless/02100_replaceRegexpAll_bug.sql | 17 ++++++++++++++--- 2 files changed, 25 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference index 2bb40778ca6..993dd9b1cde 100644 --- a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference @@ -1,3 +1,11 @@ -aaaabb -b aaaa -aaaa +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql index d0caeacfa0e..32f7f63f6d0 100644 --- a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql @@ -1,3 +1,14 @@ -SELECT trim(leading 'b ' FROM 'b aaaabb ') x; -SELECT trim(trailing 'b ' FROM 'b aaaabb ') x; -SELECT trim(both 'b ' FROM 'b aaaabb ') x; +SELECT 'aaaabb ' == trim(leading 'b ' FROM 'b aaaabb ') x; +SELECT 'b aaaa' == trim(trailing 'b ' FROM 'b aaaabb ') x; +SELECT 'aaaa' == trim(both 'b ' FROM 'b aaaabb ') x; + +SELECT '1' == replaceRegexpAll(',,1,,', '^[,]*|[,]*$', '') x; +SELECT '1' == replaceRegexpAll(',,1', '^[,]*|[,]*$', '') x; +SELECT '1' == replaceRegexpAll('1,,', '^[,]*|[,]*$', '') x; + +SELECT '1,,' == replaceRegexpOne(',,1,,', '^[,]*|[,]*$', '') x; +SELECT '1' == replaceRegexpOne(',,1', '^[,]*|[,]*$', '') x; +SELECT '1,,' == replaceRegexpOne('1,,', '^[,]*|[,]*$', '') x; + +SELECT '5935,5998,6014' == trim(BOTH ', ' FROM '5935,5998,6014, ') x; +SELECT '5935,5998,6014' == replaceRegexpAll('5935,5998,6014, ', concat('^[', regexpQuoteMeta(', '), ']*|[', regexpQuoteMeta(', '), ']*$'), '') AS x; From 36bb4033ba9a0f8dc49a6ae1f604167e284e4d67 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Oct 2021 12:44:55 +0300 Subject: [PATCH 834/950] Whitespace change in kerberized_hadoop/Dockerfile --- docker/test/integration/kerberized_hadoop/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/integration/kerberized_hadoop/Dockerfile b/docker/test/integration/kerberized_hadoop/Dockerfile index 11da590f901..4a2a8866b8d 100644 --- a/docker/test/integration/kerberized_hadoop/Dockerfile +++ b/docker/test/integration/kerberized_hadoop/Dockerfile @@ -1,6 +1,7 @@ # docker build -t clickhouse/kerberized-hadoop . FROM sequenceiq/hadoop-docker:2.7.0 + RUN sed -i -e 's/^\#baseurl/baseurl/' /etc/yum.repos.d/CentOS-Base.repo RUN sed -i -e 's/^mirrorlist/#mirrorlist/' /etc/yum.repos.d/CentOS-Base.repo RUN sed -i -e 's#http://mirror.centos.org/#http://vault.centos.org/#' /etc/yum.repos.d/CentOS-Base.repo From a92dc0a8260cc2436f098ce31cae6c5b0bdc5e03 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 12:58:10 +0300 Subject: [PATCH 835/950] Update obsolete comments. --- src/Formats/FormatFactory.h | 2 +- src/Formats/NativeReader.cpp | 1 - src/IO/Progress.h | 2 +- src/Interpreters/Aggregator.h | 2 -- src/Interpreters/Context.h | 4 ++-- src/Processors/Formats/IOutputFormat.h | 3 ++- src/Processors/Sources/SourceWithProgress.cpp | 9 +++------ src/QueryPipeline/ProfileInfo.h | 2 +- src/QueryPipeline/QueryPipelineBuilder.h | 1 - 9 files changed, 10 insertions(+), 16 deletions(-) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d5784219c6a..ee3824081bb 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -42,7 +42,7 @@ FormatSettings getFormatSettings(ContextPtr context); template FormatSettings getFormatSettings(ContextPtr context, const T & settings); -/** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format. +/** Allows to create an IInputFormat or IOutputFormat by the name of the format. * Note: format and compression are independent things. */ class FormatFactory final : private boost::noncopyable diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 2d8fdc160f5..9ef248dc904 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -56,7 +56,6 @@ NativeReader::NativeReader(ReadBuffer & istr_, UInt64 server_revision_, } } -// also resets few vars from IBlockInputStream (I didn't want to propagate resetParser upthere) void NativeReader::resetParser() { istr_concrete = nullptr; diff --git a/src/IO/Progress.h b/src/IO/Progress.h index 7118de844f2..c00eea98ff4 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -121,7 +121,7 @@ struct Progress /** Callback to track the progress of the query. - * Used in IBlockInputStream and Context. + * Used in QueryPipeline and Context. * The function takes the number of rows in the last block, the number of bytes in the last block. * Note that the callback can be called from different threads. */ diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 6d6bf61834b..3c53769e128 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -43,8 +43,6 @@ namespace ErrorCodes extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; } -class IBlockOutputStream; - /** Different data structures that can be used for aggregation * For efficiency, the aggregation data itself is put into the pool. * Data and pool ownership (states of aggregate functions) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5a28e3fac97..c6bb266120a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -636,13 +636,13 @@ public: const Settings & getSettingsRef() const { return settings; } void setProgressCallback(ProgressCallback callback); - /// Used in InterpreterSelectQuery to pass it to the IBlockInputStream. + /// Used in executeQuery() to pass it to the QueryPipeline. ProgressCallback getProgressCallback() const; void setFileProgressCallback(FileProgressCallback && callback) { file_progress_callback = callback; } FileProgressCallback getFileProgressCallback() const { return file_progress_callback; } - /** Set in executeQuery and InterpreterSelectQuery. Then it is used in IBlockInputStream, + /** Set in executeQuery and InterpreterSelectQuery. Then it is used in QueryPipeline, * to update and monitor information about the total number of resources spent for the query. */ void setProcessListElement(QueryStatus * elem); diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index b647338d6fb..ba4dcee6f70 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -72,7 +72,8 @@ public: InputPort & getPort(PortKind kind) { return *std::next(inputs.begin(), kind); } - /// Compatible to IBlockOutputStream interface + /// Compatibility with old interface. + /// TODO: separate formats and processors. void write(const Block & block); diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index 15d64dee3ee..0ebdd968997 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -69,8 +69,7 @@ void SourceWithProgress::work() } } -/// Aggregated copy-paste from IBlockInputStream::progressImpl. -/// Most of this must be done in PipelineExecutor outside. Now it's done for compatibility with IBlockInputStream. +/// TODO: Most of this must be done in PipelineExecutor outside. void SourceWithProgress::progress(const Progress & value) { was_progress_called = true; @@ -135,14 +134,12 @@ void SourceWithProgress::progress(const Progress & value) if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) { - /// Should be done in PipelineExecutor. - /// It is here for compatibility with IBlockInputsStream. + /// TODO: Should be done in PipelineExecutor. CurrentThread::updatePerformanceCounters(); last_profile_events_update_time = total_elapsed_microseconds; } - /// Should be done in PipelineExecutor. - /// It is here for compatibility with IBlockInputsStream. + /// TODO: Should be done in PipelineExecutor. limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); if (quota && limits.mode == LimitsMode::LIMITS_TOTAL) diff --git a/src/QueryPipeline/ProfileInfo.h b/src/QueryPipeline/ProfileInfo.h index 335092ce244..0a5800cd409 100644 --- a/src/QueryPipeline/ProfileInfo.h +++ b/src/QueryPipeline/ProfileInfo.h @@ -12,7 +12,7 @@ class Block; class ReadBuffer; class WriteBuffer; -/// Information for profiling. See IBlockInputStream.h +/// Information for profiling. See SourceWithProgress.h struct ProfileInfo { bool started = false; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index d2bbea03ce5..12f74805173 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -129,7 +129,6 @@ public: void setLeafLimits(const SizeLimits & limits) { pipe.setLeafLimits(limits); } void setQuota(const std::shared_ptr & quota) { pipe.setQuota(quota); } - /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); void setProcessListElement(QueryStatus * elem); From cfa685c29ca99679eb2130e99afafc53e9c3b9e7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 13:16:30 +0300 Subject: [PATCH 836/950] Fix another test. --- .../0_stateless/01823_explain_json.reference | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/tests/queries/0_stateless/01823_explain_json.reference b/tests/queries/0_stateless/01823_explain_json.reference index 5c7845a22d5..f75cd69dbf3 100644 --- a/tests/queries/0_stateless/01823_explain_json.reference +++ b/tests/queries/0_stateless/01823_explain_json.reference @@ -111,31 +111,3 @@ } ], "Limit": 3, --- - "Sort Description": [ - { - "Column": "number", - "Ascending": false, - "With Fill": false - }, - { - "Column": "plus(number, 1)", - "Ascending": true, - "With Fill": false - } - ], - "Limit": 3, --- - "Sort Description": [ - { - "Column": "number", - "Ascending": false, - "With Fill": false - }, - { - "Column": "plus(number, 1)", - "Ascending": true, - "With Fill": false - } - ], - "Limit": 3, From 4fbd332bf1bc0db29dee09699c4c737bfd2e64b0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Oct 2021 11:53:55 +0000 Subject: [PATCH 837/950] Done --- .../sql-reference/statements/select/from.md | 2 +- .../sql-reference/statements/select/from.md | 2 +- .../sql-reference/statements/select/from.md | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 20 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 6 +----- .../queries/0_stateless/01236_graphite_mt.sql | 4 +--- 6 files changed, 16 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md index 7c5ea732122..df30a0fb0d2 100644 --- a/docs/en/sql-reference/statements/select/from.md +++ b/docs/en/sql-reference/statements/select/from.md @@ -20,7 +20,7 @@ Subquery is another `SELECT` query that may be specified in parenthesis inside ` When `FINAL` is specified, ClickHouse fully merges the data before returning the result and thus performs all data transformations that happen during merges for the given table engine. -It is applicable when selecting data from tables that use the [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-engine family (except `GraphiteMergeTree`). Also supported for: +It is applicable when selecting data from tables that use the [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-engine family. Also supported for: - [Replicated](../../../engines/table-engines/mergetree-family/replication.md) versions of `MergeTree` engines. - [View](../../../engines/table-engines/special/view.md), [Buffer](../../../engines/table-engines/special/buffer.md), [Distributed](../../../engines/table-engines/special/distributed.md), and [MaterializedView](../../../engines/table-engines/special/materializedview.md) engines that operate over other engines, provided they were created over `MergeTree`-engine tables. diff --git a/docs/ru/sql-reference/statements/select/from.md b/docs/ru/sql-reference/statements/select/from.md index b62b2fd51d4..0711d602cd1 100644 --- a/docs/ru/sql-reference/statements/select/from.md +++ b/docs/ru/sql-reference/statements/select/from.md @@ -20,7 +20,7 @@ toc_title: FROM Если в запросе используется модификатор `FINAL`, то ClickHouse полностью мёржит данные перед выдачей результата, таким образом выполняя все преобразования данных, которые производятся движком таблиц при мёржах. -Он применим при выборе данных из таблиц, использующих [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)- семейство движков (кроме `GraphiteMergeTree`). Также поддерживается для: +Он применим при выборе данных из таблиц, использующих [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)- семейство движков. Также поддерживается для: - [Replicated](../../../engines/table-engines/mergetree-family/replication.md) варианты исполнения `MergeTree` движков. - [View](../../../engines/table-engines/special/view.md), [Buffer](../../../engines/table-engines/special/buffer.md), [Distributed](../../../engines/table-engines/special/distributed.md), и [MaterializedView](../../../engines/table-engines/special/materializedview.md), которые работают поверх других движков, если они созданы для таблиц с движками семейства `MergeTree`. diff --git a/docs/zh/sql-reference/statements/select/from.md b/docs/zh/sql-reference/statements/select/from.md index fae25c0c3c1..c47e74e5e1f 100644 --- a/docs/zh/sql-reference/statements/select/from.md +++ b/docs/zh/sql-reference/statements/select/from.md @@ -20,7 +20,7 @@ toc_title: FROM 当 `FINAL` 被指定,ClickHouse会在返回结果之前完全合并数据,从而执行给定表引擎合并期间发生的所有数据转换。 -它适用于从使用 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-引擎族(除了 `GraphiteMergeTree`). 还支持: +它适用于从使用 [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)-引擎族. 还支持: - [Replicated](../../../engines/table-engines/mergetree-family/replication.md) 版本 `MergeTree` 引擎 - [View](../../../engines/table-engines/special/view.md), [Buffer](../../../engines/table-engines/special/buffer.md), [Distributed](../../../engines/table-engines/special/distributed.md),和 [MaterializedView](../../../engines/table-engines/special/materializedview.md) 在其他引擎上运行的引擎,只要是它们底层是 `MergeTree`-引擎表即可。 diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 57785a5cc2d..3b1d7254e2c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -506,38 +507,39 @@ static void addMergingFinal( const auto & header = pipe.getHeader(); size_t num_outputs = pipe.numOutputPorts(); + auto now = time(nullptr); + auto get_merging_processor = [&]() -> MergingTransformPtr { switch (merging_params.mode) { case MergeTreeData::MergingParams::Ordinary: - { return std::make_shared(header, num_outputs, - sort_description, max_block_size); - } + sort_description, max_block_size); case MergeTreeData::MergingParams::Collapsing: return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, true, max_block_size); + sort_description, merging_params.sign_column, true, max_block_size); case MergeTreeData::MergingParams::Summing: return std::make_shared(header, num_outputs, - sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); + sort_description, merging_params.columns_to_sum, partition_key_columns, max_block_size); case MergeTreeData::MergingParams::Aggregating: return std::make_shared(header, num_outputs, - sort_description, max_block_size); + sort_description, max_block_size); case MergeTreeData::MergingParams::Replacing: return std::make_shared(header, num_outputs, - sort_description, merging_params.version_column, max_block_size); + sort_description, merging_params.version_column, max_block_size); case MergeTreeData::MergingParams::VersionedCollapsing: return std::make_shared(header, num_outputs, - sort_description, merging_params.sign_column, max_block_size); + sort_description, merging_params.sign_column, max_block_size); case MergeTreeData::MergingParams::Graphite: - throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); + return std::make_shared(header, num_outputs, + sort_description, max_block_size, merging_params.graphite_params, now); } __builtin_unreachable(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c4536c463d5..d20d0024222 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -402,11 +402,7 @@ public: bool supportsFinal() const override { - return merging_params.mode == MergingParams::Collapsing - || merging_params.mode == MergingParams::Summing - || merging_params.mode == MergingParams::Aggregating - || merging_params.mode == MergingParams::Replacing - || merging_params.mode == MergingParams::VersionedCollapsing; + return merging_params.mode != MergingParams::Ordinary; } bool supportsSubcolumns() const override { return true; } diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index a6dd4b8b6fb..3697a1d01d8 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -32,8 +32,6 @@ WITH dates AS select 1, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all select 2, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200); -optimize table test_graphite final; - -select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; +select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc final; drop table test_graphite; From 1114d06bc0191bc2b204cfeae3aa23ac6673c610 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Oct 2021 12:11:18 +0000 Subject: [PATCH 838/950] Done --- src/Functions/readWkt.cpp | 32 ++++++------- src/Functions/registerFunctionsGeo.cpp | 4 +- src/Functions/svg.cpp | 1 + tests/fuzz/all.dict | 9 ++-- tests/fuzz/dictionaries/functions.dict | 9 ++-- tests/queries/0_stateless/01300_read_wkt.sql | 14 +++--- tests/queries/0_stateless/01300_svg.sql | 48 ++++++++++---------- 7 files changed, 60 insertions(+), 57 deletions(-) diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index 14e12fb310c..c3ae6516e0f 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -18,10 +18,10 @@ namespace ErrorCodes template -class FunctionReadWkt : public IFunction +class FunctionReadWKT : public IFunction { public: - explicit FunctionReadWkt() = default; + explicit FunctionReadWKT() = default; static constexpr const char * name = NameHolder::name; @@ -72,36 +72,36 @@ public: static FunctionPtr create(ContextPtr) { - return std::make_shared>(); + return std::make_shared>(); } }; -struct ReadWktPointNameHolder +struct ReadWKTPointNameHolder { - static constexpr const char * name = "readWktPoint"; + static constexpr const char * name = "readWKTPoint"; }; -struct ReadWktRingNameHolder +struct ReadWKTRingNameHolder { - static constexpr const char * name = "readWktRing"; + static constexpr const char * name = "readWKTRing"; }; -struct ReadWktPolygonNameHolder +struct ReadWKTPolygonNameHolder { - static constexpr const char * name = "readWktPolygon"; + static constexpr const char * name = "readWKTPolygon"; }; -struct ReadWktMultiPolygonNameHolder +struct ReadWKTMultiPolygonNameHolder { - static constexpr const char * name = "readWktMultiPolygon"; + static constexpr const char * name = "readWKTMultiPolygon"; }; -void registerFunctionReadWkt(FunctionFactory & factory) +void registerFunctionReadWKT(FunctionFactory & factory) { - factory.registerFunction, ReadWktPointNameHolder>>(); - factory.registerFunction, ReadWktRingNameHolder>>(); - factory.registerFunction, ReadWktPolygonNameHolder>>(); - factory.registerFunction, ReadWktMultiPolygonNameHolder>>(); + factory.registerFunction, ReadWKTPointNameHolder>>(); + factory.registerFunction, ReadWKTRingNameHolder>>(); + factory.registerFunction, ReadWKTPolygonNameHolder>>(); + factory.registerFunction, ReadWKTMultiPolygonNameHolder>>(); } } diff --git a/src/Functions/registerFunctionsGeo.cpp b/src/Functions/registerFunctionsGeo.cpp index a0ae38f6b85..fd55c9cc20a 100644 --- a/src/Functions/registerFunctionsGeo.cpp +++ b/src/Functions/registerFunctionsGeo.cpp @@ -23,7 +23,7 @@ void registerFunctionGeohashEncode(FunctionFactory & factory); void registerFunctionGeohashDecode(FunctionFactory & factory); void registerFunctionGeohashesInBox(FunctionFactory & factory); void registerFunctionWkt(FunctionFactory & factory); -void registerFunctionReadWkt(FunctionFactory & factory); +void registerFunctionReadWKT(FunctionFactory & factory); void registerFunctionSvg(FunctionFactory & factory); #if USE_H3 @@ -79,7 +79,7 @@ void registerFunctionsGeo(FunctionFactory & factory) registerFunctionGeohashDecode(factory); registerFunctionGeohashesInBox(factory); registerFunctionWkt(factory); - registerFunctionReadWkt(factory); + registerFunctionReadWKT(factory); registerFunctionSvg(factory); #if USE_H3 diff --git a/src/Functions/svg.cpp b/src/Functions/svg.cpp index 4495e668add..b3a89c0393c 100644 --- a/src/Functions/svg.cpp +++ b/src/Functions/svg.cpp @@ -102,6 +102,7 @@ public: void registerFunctionSvg(FunctionFactory & factory) { factory.registerFunction(); + factory.registerAlias("SVG", "svg"); } } diff --git a/tests/fuzz/all.dict b/tests/fuzz/all.dict index 4a9afc348cf..bf25f1fa484 100644 --- a/tests/fuzz/all.dict +++ b/tests/fuzz/all.dict @@ -985,10 +985,10 @@ "RANGE" "rank" "rankCorr" -"readWktMultiPolygon" -"readWktPoint" -"readWktPolygon" -"readWktRing" +"readWKTMultiPolygon" +"readWKTPoint" +"readWKTPolygon" +"readWKTRing" "REAL" "REFRESH" "regexpQuoteMeta" @@ -1177,6 +1177,7 @@ "sumWithOverflow" "SUSPEND" "svg" +"SVG" "SYNC" "synonyms" "SYNTAX" diff --git a/tests/fuzz/dictionaries/functions.dict b/tests/fuzz/dictionaries/functions.dict index fb35375f284..722e931dc09 100644 --- a/tests/fuzz/dictionaries/functions.dict +++ b/tests/fuzz/dictionaries/functions.dict @@ -52,6 +52,7 @@ "h3GetResolution" "h3EdgeLengthM" "svg" +"SVG" "equals" "geohashesInBox" "polygonsIntersectionCartesian" @@ -114,7 +115,7 @@ "replaceOne" "emptyArrayInt32" "extract" -"readWktPolygon" +"readWKTPolygon" "notILike" "geohashDecode" "toModifiedJulianDay" @@ -164,7 +165,7 @@ "lessOrEquals" "subtractQuarters" "ngramSearch" -"readWktRing" +"readWKTRing" "trimRight" "endsWith" "ngramDistanceCaseInsensitive" @@ -713,13 +714,13 @@ "s2RectContains" "toDate" "regexpQuoteMeta" -"readWktMultiPolygon" +"readWKTMultiPolygon" "emptyArrayString" "bitmapOr" "cutWWW" "emptyArrayInt8" "less" -"readWktPoint" +"readWKTPoint" "reinterpretAsDateTime" "notEquals" "geoToS2" diff --git a/tests/queries/0_stateless/01300_read_wkt.sql b/tests/queries/0_stateless/01300_read_wkt.sql index 8121bdf6084..1995c5153d7 100644 --- a/tests/queries/0_stateless/01300_read_wkt.sql +++ b/tests/queries/0_stateless/01300_read_wkt.sql @@ -1,14 +1,14 @@ -SELECT readWktPoint('POINT(0 0)'); -SELECT readWktPolygon('POLYGON((1 0,10 0,10 10,0 10,1 0))'); -SELECT readWktPolygon('POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))'); -SELECT readWktMultiPolygon('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))'); +SELECT readWKTPoint('POINT(0 0)'); +SELECT readWKTPolygon('POLYGON((1 0,10 0,10 10,0 10,1 0))'); +SELECT readWKTPolygon('POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))'); +SELECT readWKTMultiPolygon('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))'); DROP TABLE IF EXISTS geo; CREATE TABLE geo (s String, id Int) engine=Memory(); INSERT INTO geo VALUES ('POINT(0 0)', 1); INSERT INTO geo VALUES ('POINT(1 0)', 2); INSERT INTO geo VALUES ('POINT(2 0)', 3); -SELECT readWktPoint(s) FROM geo ORDER BY id; +SELECT readWKTPoint(s) FROM geo ORDER BY id; DROP TABLE IF EXISTS geo; CREATE TABLE geo (s String, id Int) engine=Memory(); @@ -18,13 +18,13 @@ INSERT INTO geo VALUES ('POLYGON((2 0,10 0,10 10,0 10,2 0))', 3); INSERT INTO geo VALUES ('POLYGON((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4))', 4); INSERT INTO geo VALUES ('POLYGON((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4))', 5); INSERT INTO geo VALUES ('POLYGON((1 0,10 0,10 10,0 10,1 0),(4 4,5 4,5 5,4 5,4 4))', 6); -SELECT readWktPolygon(s) FROM geo ORDER BY id; +SELECT readWKTPolygon(s) FROM geo ORDER BY id; DROP TABLE IF EXISTS geo; CREATE TABLE geo (s String, id Int) engine=Memory(); INSERT INTO geo VALUES ('MULTIPOLYGON(((1 0,10 0,10 10,0 10,1 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))', 1); INSERT INTO geo VALUES ('MULTIPOLYGON(((0 0,10 0,10 10,0 10,0 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))', 2); INSERT INTO geo VALUES ('MULTIPOLYGON(((2 0,10 0,10 10,0 10,2 0),(4 4,5 4,5 5,4 5,4 4)),((-10 -10,-10 -9,-9 10,-10 -10)))', 3); -SELECT readWktMultiPolygon(s) FROM geo ORDER BY id; +SELECT readWKTMultiPolygon(s) FROM geo ORDER BY id; DROP TABLE geo; diff --git a/tests/queries/0_stateless/01300_svg.sql b/tests/queries/0_stateless/01300_svg.sql index a1deb1745c3..cf794f2190b 100644 --- a/tests/queries/0_stateless/01300_svg.sql +++ b/tests/queries/0_stateless/01300_svg.sql @@ -1,50 +1,50 @@ -SELECT svg((0., 0.)); -SELECT svg([(0., 0.), (10, 0), (10, 10), (0, 10)]); -SELECT svg([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]]); -SELECT svg([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]]); -SELECT svg((0., 0.), 'b'); -SELECT svg([(0., 0.), (10, 0), (10, 10), (0, 10)], 'b'); -SELECT svg([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], 'b'); -SELECT svg([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'b'); +SELECT SVG((0., 0.)); +SELECT SVG([(0., 0.), (10, 0), (10, 10), (0, 10)]); +SELECT SVG([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]]); +SELECT SVG([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]]); +SELECT SVG((0., 0.), 'b'); +SELECT SVG([(0., 0.), (10, 0), (10, 10), (0, 10)], 'b'); +SELECT SVG([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], 'b'); +SELECT SVG([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'b'); DROP TABLE IF EXISTS geo; CREATE TABLE geo (p Tuple(Float64, Float64), s String, id Int) engine=Memory(); INSERT INTO geo VALUES ((0., 0.), 'b', 1); INSERT INTO geo VALUES ((1., 0.), 'c', 2); INSERT INTO geo VALUES ((2., 0.), 'd', 3); -SELECT svg(p) FROM geo ORDER BY id; -SELECT svg(p, 'b') FROM geo ORDER BY id; -SELECT svg((0., 0.), s) FROM geo ORDER BY id; -SELECT svg(p, s) FROM geo ORDER BY id; +SELECT SVG(p) FROM geo ORDER BY id; +SELECT SVG(p, 'b') FROM geo ORDER BY id; +SELECT SVG((0., 0.), s) FROM geo ORDER BY id; +SELECT SVG(p, s) FROM geo ORDER BY id; DROP TABLE IF EXISTS geo; CREATE TABLE geo (p Array(Tuple(Float64, Float64)), s String, id Int) engine=Memory(); INSERT INTO geo VALUES ([(0., 0.), (10, 0), (10, 10), (0, 10)], 'b', 1); INSERT INTO geo VALUES ([(1., 0.), (10, 0), (10, 10), (0, 10)], 'c', 2); INSERT INTO geo VALUES ([(2., 0.), (10, 0), (10, 10), (0, 10)], 'd', 3); -SELECT svg(p) FROM geo ORDER BY id; -SELECT svg(p, 'b') FROM geo ORDER BY id; -SELECT svg([(0., 0.), (10, 0), (10, 10), (0, 10)], s) FROM geo ORDER BY id; -SELECT svg(p, s) FROM geo ORDER BY id; +SELECT SVG(p) FROM geo ORDER BY id; +SELECT SVG(p, 'b') FROM geo ORDER BY id; +SELECT SVG([(0., 0.), (10, 0), (10, 10), (0, 10)], s) FROM geo ORDER BY id; +SELECT SVG(p, s) FROM geo ORDER BY id; DROP TABLE IF EXISTS geo; CREATE TABLE geo (p Array(Array(Tuple(Float64, Float64))), s String, id Int) engine=Memory(); INSERT INTO geo VALUES ([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 'b', 1); INSERT INTO geo VALUES ([[(1., 0.), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 'c', 2); INSERT INTO geo VALUES ([[(2., 0.), (10, 0), (10, 10), (0, 10)], [(4, 4), (5, 4), (5, 5), (4, 5)]], 'd', 3); -SELECT svg(p) FROM geo ORDER BY id; -SELECT svg(p, 'b') FROM geo ORDER BY id; -SELECT svg([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], s) FROM geo ORDER BY id; -SELECT svg(p, s) FROM geo ORDER BY id; +SELECT SVG(p) FROM geo ORDER BY id; +SELECT SVG(p, 'b') FROM geo ORDER BY id; +SELECT SVG([[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], s) FROM geo ORDER BY id; +SELECT SVG(p, s) FROM geo ORDER BY id; DROP TABLE IF EXISTS geo; CREATE TABLE geo (p Array(Array(Array(Tuple(Float64, Float64)))), s String, id Int) engine=Memory(); INSERT INTO geo VALUES ([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'b', 1); INSERT INTO geo VALUES ([[[(1., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'c', 2); INSERT INTO geo VALUES ([[[(2., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], 'd', 3); -SELECT svg(p) FROM geo ORDER BY id; -SELECT svg(p, 'b') FROM geo ORDER BY id; -SELECT svg([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], s) FROM geo ORDER BY id; -SELECT svg(p, s) FROM geo ORDER BY id; +SELECT SVG(p) FROM geo ORDER BY id; +SELECT SVG(p, 'b') FROM geo ORDER BY id; +SELECT SVG([[[(0., 0.), (10, 0), (10, 10), (0, 10)], [(4., 4.), (5, 4), (5, 5), (4, 5)]], [[(-10., -10.), (-10, -9), (-9, 10)]]], s) FROM geo ORDER BY id; +SELECT SVG(p, s) FROM geo ORDER BY id; DROP TABLE geo; From 75487be8998d02b22a800e27463a7b942053a80d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Oct 2021 12:16:33 +0000 Subject: [PATCH 839/950] White list of storages that supports final --- src/Storages/MergeTree/MergeTreeData.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d20d0024222..2ea6a89002c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -402,7 +402,12 @@ public: bool supportsFinal() const override { - return merging_params.mode != MergingParams::Ordinary; + return merging_params.mode == MergingParams::Collapsing + || merging_params.mode == MergingParams::Summing + || merging_params.mode == MergingParams::Aggregating + || merging_params.mode == MergingParams::Replacing + || merging_params.mode == MergingParams::Graphite + || merging_params.mode == MergingParams::VersionedCollapsing; } bool supportsSubcolumns() const override { return true; } From 08f3a01830867cac67703cd680a1c7280a44a079 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Oct 2021 16:26:16 +0300 Subject: [PATCH 840/950] Update 01236_graphite_mt.sql --- tests/queries/0_stateless/01236_graphite_mt.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 3697a1d01d8..1d531f88ecb 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -32,6 +32,6 @@ WITH dates AS select 1, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200) union all select 2, 'max_2', older_date - number * 60 - 30, number, 1, number from dates, numbers(1200); -select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc final; +select key, Path, Value, Version, col from test_graphite final order by key, Path, Time desc; drop table test_graphite; From c547ffa31395b15dcbfadeaafcb26941269c895b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 16:36:31 +0300 Subject: [PATCH 841/950] improve performance of aggregation in order primary key with function with heavy states --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../FinishAggregatingInOrderAlgorithm.cpp | 37 +++++-- .../FinishAggregatingInOrderAlgorithm.h | 15 ++- .../FinishAggregatingInOrderTransform.h | 6 +- src/Processors/QueryPlan/AggregatingStep.cpp | 17 ++- src/Processors/QueryPlan/AggregatingStep.h | 2 + .../AggregatingInOrderTransform.cpp | 103 +++++++++++------- .../Transforms/AggregatingInOrderTransform.h | 19 +++- tests/performance/aggregation_in_order_2.xml | 30 +++++ 10 files changed, 167 insertions(+), 64 deletions(-) create mode 100644 tests/performance/aggregation_in_order_2.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a5767955045..6bd37432c2c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -386,6 +386,7 @@ class IColumn; M(Bool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \ M(Bool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ M(Bool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \ + M(UInt64, aggregation_in_order_max_block_bytes, 50000000, "Maximal size of block in bytes accumulated during aggregation in order of primary key. Lower block size allows to parallelize more final merge stage of aggregation.", 0) \ M(UInt64, read_in_order_two_level_merge_threshold, 100, "Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.", 0) \ M(Bool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \ M(Bool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85cc889319f..a0432607af5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2105,6 +2105,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac params, final, settings.max_block_size, + settings.aggregation_in_order_max_block_bytes, merge_threads, temporary_data_merge_threads, storage_has_evenly_distributed_read, diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index f54331d5550..5fc57fb42d8 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -9,9 +10,10 @@ namespace DB { FinishAggregatingInOrderAlgorithm::State::State( - const Chunk & chunk, const SortDescription & desc) - : num_rows(chunk.getNumRows()) - , all_columns(chunk.getColumns()) + const Chunk & chunk, const SortDescription & desc, Int64 total_bytes_) + : all_columns(chunk.getColumns()) + , num_rows(chunk.getNumRows()) + , total_bytes(total_bytes_) { if (!chunk) return; @@ -26,12 +28,14 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_) + size_t max_block_size_, + size_t max_block_bytes_) : header(header_) , num_inputs(num_inputs_) , params(params_) , description(std::move(description_)) , max_block_size(max_block_size_) + , max_block_bytes(max_block_bytes_) { /// Replace column names in description to positions. for (auto & column_description : description) @@ -47,14 +51,22 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( void FinishAggregatingInOrderAlgorithm::initialize(Inputs inputs) { current_inputs = std::move(inputs); - states.reserve(num_inputs); + states.resize(num_inputs); for (size_t i = 0; i < num_inputs; ++i) - states.emplace_back(current_inputs[i].chunk, description); + consume(current_inputs[i], i); } void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num) { - states[source_num] = State{input.chunk, description}; + const auto & info = input.chunk.getChunkInfo(); + if (!info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in FinishAggregatingInOrderAlgorithm"); + + const auto * arenas_info = typeid_cast(info.get()); + if (!arenas_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have ChunkInfoWithAllocatedBytes in FinishAggregatingInOrderAlgorithm"); + + states[source_num] = State{input.chunk, description, arenas_info->allocated_bytes}; } IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() @@ -111,8 +123,8 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() Status status(inputs_to_update.back()); inputs_to_update.pop_back(); - /// Do not merge blocks, if there are too few rows. - if (accumulated_rows >= max_block_size) + /// Do not merge blocks, if there are too few rows or bytes. + if (accumulated_rows >= max_block_size || accumulated_bytes >= max_block_bytes) status.chunk = prepareToMerge(); return status; @@ -121,6 +133,8 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() { accumulated_rows = 0; + accumulated_bytes = 0; + auto info = std::make_shared(); info->chunks = std::make_unique(std::move(chunks)); @@ -153,10 +167,13 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() } chunks.back().setChunkInfo(std::make_shared()); - states[i].current_row = states[i].to_row; + + /// We assume that sizes in bytes of rows are almost the same. + accumulated_bytes += states[i].total_bytes * (static_cast(current_rows) / states[i].num_rows); accumulated_rows += current_rows; + if (!states[i].isValid()) inputs_to_update.push_back(i); } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 209bb9e4045..f3a1bd40635 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -42,7 +42,8 @@ public: size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_); + size_t max_block_size_, + size_t max_block_bytes_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; @@ -54,17 +55,23 @@ private: struct State { - size_t num_rows; Columns all_columns; ColumnRawPtrs sorting_columns; + size_t num_rows = 0; + /// Number of row starting from which need to aggregate. size_t current_row = 0; /// Number of row up to which need to aggregate (not included). size_t to_row = 0; - State(const Chunk & chunk, const SortDescription & description); + /// Number of bytes in all columns + number of bytes in arena, related to current chunk. + size_t total_bytes = 0; + + State(const Chunk & chunk, const SortDescription & description, Int64 total_bytes_); + State() = default; + bool isValid() const { return current_row < num_rows; } }; @@ -73,6 +80,7 @@ private: AggregatingTransformParamsPtr params; SortDescription description; size_t max_block_size; + size_t max_block_bytes; Inputs current_inputs; @@ -81,6 +89,7 @@ private: std::vector chunks; size_t accumulated_rows = 0; + size_t accumulated_bytes = 0; }; } diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 5605cea8e39..5dd875c8412 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -17,14 +17,16 @@ public: size_t num_inputs, AggregatingTransformParamsPtr params, SortDescription description, - size_t max_block_size) + size_t max_block_size, + size_t max_block_bytes) : IMergingTransform( num_inputs, header, {}, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, header, num_inputs, params, std::move(description), - max_block_size) + max_block_size, + max_block_bytes) { } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 51bd65a209b..219d5efa027 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -30,6 +30,7 @@ AggregatingStep::AggregatingStep( Aggregator::Params params_, bool final_, size_t max_block_size_, + size_t aggregation_in_order_max_block_bytes_, size_t merge_threads_, size_t temporary_data_merge_threads_, bool storage_has_evenly_distributed_read_, @@ -39,6 +40,7 @@ AggregatingStep::AggregatingStep( , params(std::move(params_)) , final(std::move(final_)) , max_block_size(max_block_size_) + , aggregation_in_order_max_block_bytes(aggregation_in_order_max_block_bytes_) , merge_threads(merge_threads_) , temporary_data_merge_threads(temporary_data_merge_threads_) , storage_has_evenly_distributed_read(storage_has_evenly_distributed_read_) @@ -90,7 +92,13 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B size_t counter = 0; pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, transform_params, group_by_sort_description, max_block_size, many_data, counter++); + /// We want to merge aggregated data in batches of size + /// not greater than 'aggregation_in_order_max_block_bytes'. + /// So, we reduce 'max_bytes' value for aggregation in 'merge_threads' times. + return std::make_shared( + header, transform_params, group_by_sort_description, + max_block_size, aggregation_in_order_max_block_bytes / merge_threads, + many_data, counter++); }); aggregating_in_order = collector.detachProcessors(0); @@ -100,7 +108,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B pipeline.getNumStreams(), transform_params, group_by_sort_description, - max_block_size); + max_block_size, + aggregation_in_order_max_block_bytes); pipeline.addTransform(std::move(transform)); @@ -118,7 +127,9 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B { pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, transform_params, group_by_sort_description, max_block_size); + return std::make_shared( + header, transform_params, group_by_sort_description, + max_block_size, aggregation_in_order_max_block_bytes); }); pipeline.addSimpleTransform([&](const Block & header) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 8583e5be485..1a4f36a6711 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -19,6 +19,7 @@ public: Aggregator::Params params_, bool final_, size_t max_block_size_, + size_t aggregation_in_order_max_block_bytes_, size_t merge_threads_, size_t temporary_data_merge_threads_, bool storage_has_evenly_distributed_read_, @@ -40,6 +41,7 @@ private: Aggregator::Params params; bool final; size_t max_block_size; + size_t aggregation_in_order_max_block_bytes; size_t merge_threads; size_t temporary_data_merge_threads; diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index d6c5491f45c..a1cae41fb9d 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -13,19 +13,24 @@ namespace ErrorCodes AggregatingInOrderTransform::AggregatingInOrderTransform( - Block header, AggregatingTransformParamsPtr params_, - const SortDescription & group_by_description_, size_t res_block_size_) - : AggregatingInOrderTransform(std::move(header), std::move(params_) - , group_by_description_, res_block_size_, std::make_unique(1), 0) + Block header, + AggregatingTransformParamsPtr params_, + const SortDescription & group_by_description_, + size_t max_block_size_, size_t max_block_bytes_) + : AggregatingInOrderTransform(std::move(header), std::move(params_), + group_by_description_, max_block_size_, max_block_bytes_, + std::make_unique(1), 0) { } AggregatingInOrderTransform::AggregatingInOrderTransform( Block header, AggregatingTransformParamsPtr params_, - const SortDescription & group_by_description_, size_t res_block_size_, + const SortDescription & group_by_description_, + size_t max_block_size_, size_t max_block_bytes_, ManyAggregatedDataPtr many_data_, size_t current_variant) : IProcessor({std::move(header)}, {params_->getCustomHeader(false)}) - , res_block_size(res_block_size_) + , max_block_size(max_block_size_) + , max_block_bytes(max_block_bytes_) , params(std::move(params_)) , group_by_description(group_by_description_) , aggregate_columns(params->params.aggregates_size) @@ -48,8 +53,18 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( AggregatingInOrderTransform::~AggregatingInOrderTransform() = default; +static Int64 getCurrentMemoryUsage() +{ + Int64 current_memory_usage = 0; + if (auto * memory_tracker = CurrentThread::getMemoryTracker()) + current_memory_usage = memory_tracker->get(); + return current_memory_usage; +} + void AggregatingInOrderTransform::consume(Chunk chunk) { + Int64 initial_memory_usage = getCurrentMemoryUsage(); + size_t rows = chunk.getNumRows(); if (rows == 0) return; @@ -77,6 +92,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) size_t key_end = 0; size_t key_begin = 0; + /// If we don't have a block we create it and fill with first key if (!cur_block_size) { @@ -94,6 +110,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) ++cur_block_size; } + Int64 current_memory_usage = 0; /// Will split block into segments with the same key while (key_end != rows) @@ -112,44 +129,18 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (key_begin != key_end) params->aggregator.executeOnIntervalWithoutKeyImpl(variants.without_key, key_begin, key_end, aggregate_function_instructions.data(), variants.aggregates_pool); + current_memory_usage = getCurrentMemoryUsage() - initial_memory_usage; + /// We finalize last key aggregation state if a new key found. if (key_end != rows) { params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); - /// If res_block_size is reached we have to stop consuming and generate the block. Save the extra rows into new chunk. - if (cur_block_size == res_block_size) + /// If max_block_size is reached we have to stop consuming and generate the block. Save the extra rows into new chunk. + if (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes) { - Columns source_columns = chunk.detachColumns(); - - for (auto & source_column : source_columns) - source_column = source_column->cut(key_end, rows - key_end); - - current_chunk = Chunk(source_columns, rows - key_end); - src_rows -= current_chunk.getNumRows(); - block_end_reached = true; - need_generate = true; - cur_block_size = 0; - - variants.without_key = nullptr; - - /// Arenas cannot be destroyed here, since later, in FinalizeAggregatedTransform - /// there will be finalizeChunk(), but even after - /// finalizeChunk() we cannot destroy arena, since some memory - /// from Arena still in use, so we attach it to the Chunk to - /// remove it once it will be consumed. - if (params->final) - { - if (variants.aggregates_pools.size() != 1) - throw Exception("Too much arenas", ErrorCodes::LOGICAL_ERROR); - - Arenas arenas(1, std::make_shared()); - std::swap(variants.aggregates_pools, arenas); - variants.aggregates_pool = variants.aggregates_pools.at(0).get(); - - chunk.setChunkInfo(std::make_shared(std::move(arenas))); - } - + cur_block_bytes += current_memory_usage; + finalizeCurrentChunk(std::move(chunk), key_end); return; } @@ -161,9 +152,25 @@ void AggregatingInOrderTransform::consume(Chunk chunk) key_begin = key_end; } + cur_block_bytes += current_memory_usage; block_end_reached = false; } +void AggregatingInOrderTransform::finalizeCurrentChunk(Chunk chunk, size_t key_end) +{ + size_t rows = chunk.getNumRows(); + Columns source_columns = chunk.detachColumns(); + + for (auto & source_column : source_columns) + source_column = source_column->cut(key_end, rows - key_end); + + current_chunk = Chunk(source_columns, rows - key_end); + src_rows -= current_chunk.getNumRows(); + + block_end_reached = true; + need_generate = true; + variants.without_key = nullptr; +} void AggregatingInOrderTransform::work() { @@ -218,6 +225,7 @@ IProcessor::Status AggregatingInOrderTransform::prepare() src_rows, res_rows, formatReadableSizeWithBinarySuffix(src_bytes)); return Status::Finished; } + if (input.isFinished()) { is_consume_finished = true; @@ -245,14 +253,25 @@ void AggregatingInOrderTransform::generate() Block res = res_header.cloneEmpty(); for (size_t i = 0; i < res_key_columns.size(); ++i) - { res.getByPosition(i).column = std::move(res_key_columns[i]); - } + for (size_t i = 0; i < res_aggregate_columns.size(); ++i) - { res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); - } + to_push_chunk = convertToChunk(res); + to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); + + /// Clear arenas to allow to free them, when chunk will reach the end of pipeline. + /// It's safe clear them here, because columns with aggregate functions already holds them. + variants.aggregates_pools = { std::make_shared() }; + variants.aggregates_pool = variants.aggregates_pools.at(0).get(); + + /// Pass info about used memory by aggregate functions further. + to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); + + cur_block_bytes = 0; + cur_block_size = 0; + res_rows += to_push_chunk.getNumRows(); need_generate = false; } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index e0cb9b80967..929ab98d6e6 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -9,16 +9,24 @@ namespace DB { +struct ChunkInfoWithAllocatedBytes : public ChunkInfo +{ + ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) + : allocated_bytes(allocated_bytes_) {} + Int64 allocated_bytes; +}; + class AggregatingInOrderTransform : public IProcessor { - public: AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, - const SortDescription & group_by_description, size_t res_block_size, + const SortDescription & group_by_description, + size_t max_block_size_, size_t max_block_bytes_, ManyAggregatedDataPtr many_data, size_t current_variant); AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, - const SortDescription & group_by_description, size_t res_block_size); + const SortDescription & group_by_description, + size_t max_block_size_, size_t max_block_bytes_); ~AggregatingInOrderTransform() override; @@ -32,9 +40,12 @@ public: private: void generate(); + void finalizeCurrentChunk(Chunk chunk, size_t key_end); - size_t res_block_size; + size_t max_block_size; + size_t max_block_bytes; size_t cur_block_size = 0; + size_t cur_block_bytes = 0; MutableColumns res_key_columns; MutableColumns res_aggregate_columns; diff --git a/tests/performance/aggregation_in_order_2.xml b/tests/performance/aggregation_in_order_2.xml new file mode 100644 index 00000000000..94a33a98c3e --- /dev/null +++ b/tests/performance/aggregation_in_order_2.xml @@ -0,0 +1,30 @@ + + + 1 + 10000000 + 30 + + + + + uniqs + + 100 + 10000 + 1000000 + + + + + CREATE TABLE mt_50_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % 50 SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 + SYSTEM STOP MERGES mt_50_parts_{uniqs}_uniqs + INSERT INTO mt_50_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 + SYSTEM START MERGES mt_50_parts_{uniqs}_uniqs + OPTIMIZE TABLE mt_50_parts_{uniqs}_uniqs FINAL + + SELECT sum(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT groupArray(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT uniqExact(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + + DROP TABLE IF EXISTS mt_50_parts_{uniqs}_uniqs + From 1d78f1c63b525812699fc16586f7fb7409162bc0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Oct 2021 16:40:06 +0300 Subject: [PATCH 842/950] Fix ca-bundle.crt in clickhouse/kerberized-hadoop --- docker/test/integration/kerberized_hadoop/Dockerfile | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/integration/kerberized_hadoop/Dockerfile b/docker/test/integration/kerberized_hadoop/Dockerfile index 4a2a8866b8d..7bc0a99f9e9 100644 --- a/docker/test/integration/kerberized_hadoop/Dockerfile +++ b/docker/test/integration/kerberized_hadoop/Dockerfile @@ -5,6 +5,10 @@ FROM sequenceiq/hadoop-docker:2.7.0 RUN sed -i -e 's/^\#baseurl/baseurl/' /etc/yum.repos.d/CentOS-Base.repo RUN sed -i -e 's/^mirrorlist/#mirrorlist/' /etc/yum.repos.d/CentOS-Base.repo RUN sed -i -e 's#http://mirror.centos.org/#http://vault.centos.org/#' /etc/yum.repos.d/CentOS-Base.repo + +# https://community.letsencrypt.org/t/rhel-centos-6-openssl-client-compatibility-after-dst-root-ca-x3-expiration/161032/81 +RUN sed -i s/xMDkzMDE0MDExNVow/0MDkzMDE4MTQwM1ow/ /etc/pki/tls/certs/ca-bundle.crt + RUN yum clean all && \ rpm --rebuilddb && \ yum -y update && \ From acf416900dbc3181859c674d13d46849e848d1bf Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Oct 2021 16:45:03 +0300 Subject: [PATCH 843/950] Minor fix in clickhouse/kerberized-hadoop Dockerfile --- docker/test/integration/kerberized_hadoop/Dockerfile | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docker/test/integration/kerberized_hadoop/Dockerfile b/docker/test/integration/kerberized_hadoop/Dockerfile index 7bc0a99f9e9..00944cbfc00 100644 --- a/docker/test/integration/kerberized_hadoop/Dockerfile +++ b/docker/test/integration/kerberized_hadoop/Dockerfile @@ -2,9 +2,9 @@ FROM sequenceiq/hadoop-docker:2.7.0 -RUN sed -i -e 's/^\#baseurl/baseurl/' /etc/yum.repos.d/CentOS-Base.repo -RUN sed -i -e 's/^mirrorlist/#mirrorlist/' /etc/yum.repos.d/CentOS-Base.repo -RUN sed -i -e 's#http://mirror.centos.org/#http://vault.centos.org/#' /etc/yum.repos.d/CentOS-Base.repo +RUN sed -i -e 's/^\#baseurl/baseurl/' /etc/yum.repos.d/CentOS-Base.repo && \ + sed -i -e 's/^mirrorlist/#mirrorlist/' /etc/yum.repos.d/CentOS-Base.repo && \ + sed -i -e 's#http://mirror.centos.org/#http://vault.centos.org/#' /etc/yum.repos.d/CentOS-Base.repo # https://community.letsencrypt.org/t/rhel-centos-6-openssl-client-compatibility-after-dst-root-ca-x3-expiration/161032/81 RUN sed -i s/xMDkzMDE0MDExNVow/0MDkzMDE4MTQwM1ow/ /etc/pki/tls/certs/ca-bundle.crt @@ -14,8 +14,9 @@ RUN yum clean all && \ yum -y update && \ yum -y install yum-plugin-ovl && \ yum --quiet -y install krb5-workstation.x86_64 + RUN cd /tmp && \ - curl http://archive.apache.org/dist/commons/daemon/source/commons-daemon-1.0.15-src.tar.gz -o commons-daemon-1.0.15-src.tar.gz && \ + curl http://archive.apache.org/dist/commons/daemon/source/commons-daemon-1.0.15-src.tar.gz -o commons-daemon-1.0.15-src.tar.gz && \ tar xzf commons-daemon-1.0.15-src.tar.gz && \ cd commons-daemon-1.0.15-src/src/native/unix && \ ./configure && \ From 8d5ed8a342a00d1d73148300ce05bf057e492133 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 17:05:54 +0300 Subject: [PATCH 844/950] remove useless line --- src/Processors/Transforms/AggregatingInOrderTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index a1cae41fb9d..e8ac969245d 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -259,7 +259,6 @@ void AggregatingInOrderTransform::generate() res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); to_push_chunk = convertToChunk(res); - to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); /// Clear arenas to allow to free them, when chunk will reach the end of pipeline. /// It's safe clear them here, because columns with aggregate functions already holds them. From 82b409e2b0dfc82de78e37a75e52fb7b6bacd1b2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 17:13:52 +0300 Subject: [PATCH 845/950] better perf test --- tests/performance/aggregation_in_order_2.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/performance/aggregation_in_order_2.xml b/tests/performance/aggregation_in_order_2.xml index 94a33a98c3e..083a8031d46 100644 --- a/tests/performance/aggregation_in_order_2.xml +++ b/tests/performance/aggregation_in_order_2.xml @@ -16,15 +16,15 @@ - CREATE TABLE mt_50_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % 50 SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 - SYSTEM STOP MERGES mt_50_parts_{uniqs}_uniqs - INSERT INTO mt_50_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 - SYSTEM START MERGES mt_50_parts_{uniqs}_uniqs - OPTIMIZE TABLE mt_50_parts_{uniqs}_uniqs FINAL + CREATE TABLE mt_30_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % 30 SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 + SYSTEM STOP MERGES mt_30_parts_{uniqs}_uniqs + INSERT INTO mt_30_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 + SYSTEM START MERGES mt_30_parts_{uniqs}_uniqs + OPTIMIZE TABLE mt_30_parts_{uniqs}_uniqs FINAL - SELECT sum(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - SELECT groupArray(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - SELECT uniqExact(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT sum(val2) FROM mt_30_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT groupArray(val2) FROM mt_30_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT uniqExact(val2) FROM mt_30_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - DROP TABLE IF EXISTS mt_50_parts_{uniqs}_uniqs + DROP TABLE IF EXISTS mt_30_parts_{uniqs}_uniqs From 0f2e23d775f39d49c32b2a0d03f61b46b468fe35 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 19 Oct 2021 17:41:59 +0300 Subject: [PATCH 846/950] Update StorageExecutable.cpp --- src/Storages/StorageExecutable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 6a82fc88977..16647d0b60f 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -122,7 +122,7 @@ Pipe StorageExecutable::read( if (!std::filesystem::exists(std::filesystem::path(script_path))) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Executable file {} does not exists inside user scripts folder {}", + "Executable file {} does not exist inside user scripts folder {}", script_name, user_scripts_path); From cef993233fa5d2deb67e64287b628f645d81fad6 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 19 Oct 2021 15:54:28 +0300 Subject: [PATCH 847/950] Attempt to fix #30162 Added some logging to the Session --- src/Interpreters/Session.cpp | 52 ++++++++++++++++++++++++++++++++++-- src/Interpreters/Session.h | 2 ++ 2 files changed, 52 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 98d0bcb88ac..75117bd860c 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -240,10 +241,34 @@ void Session::shutdownNamedSessions() NamedSessionsStorage::instance().shutdown(); } +namespace +{ +String getSessionPrefix(ClientInfo::Interface interface) +{ + switch (interface) + { + case ClientInfo::Interface::TCP: + return "TCP"; + case ClientInfo::Interface::HTTP: + return "HTTP"; + case ClientInfo::Interface::GRPC: + return "GRPC"; + case ClientInfo::Interface::MYSQL: + return "MySQL"; + case ClientInfo::Interface::POSTGRESQL: + return "PostgreSQL"; + case ClientInfo::Interface::LOCAL: + return "Local"; + case ClientInfo::Interface::TCP_INTERSERVER: + return "Interserver"; + } +} +} Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_) : session_id(UUIDHelpers::generateV4()), - global_context(global_context_) + global_context(global_context_), + log(&Poco::Logger::get(getSessionPrefix(interface_) + "-Session")) { prepared_client_info.emplace(); prepared_client_info->interface = interface_; @@ -253,6 +278,12 @@ Session::Session(Session &&) = default; Session::~Session() { + LOG_DEBUG(log, "{} Destroying {} of user {}", + toString(session_id), + (named_session ? "named session '" + named_session->key.second + "'" : "unnamed session"), + (user_id ? toString(*user_id) : "") + ); + /// Early release a NamedSessionData. if (named_session) named_session->release(); @@ -298,12 +329,18 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So if ((address == Poco::Net::SocketAddress{}) && (prepared_client_info->interface == ClientInfo::Interface::LOCAL)) address = Poco::Net::SocketAddress{"127.0.0.1", 0}; + LOG_DEBUG(log, "{} Authenticating user '{}' from {}", + toString(session_id), credentials_.getUserName(), address.toString()); + try { user_id = global_context->getAccessControlManager().login(credentials_, address.host()); + LOG_DEBUG(log, "{} Authenticated with global context as user {}", + toString(session_id), user_id ? toString(*user_id) : ""); } catch (const Exception & e) { + LOG_DEBUG(log, "{} Authentication failed with error: {}", toString(session_id), e.what()); if (auto session_log = getSessionLog()) session_log->addLoginFailure(session_id, *prepared_client_info, credentials_.getUserName(), e); throw; @@ -336,6 +373,8 @@ ContextMutablePtr Session::makeSessionContext() if (query_context_created) throw Exception("Session context must be created before any query context", ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(log, "{} Creating session context with user_id: {}", + toString(session_id), user_id ? toString(*user_id) : ""); /// Make a new session context. ContextMutablePtr new_session_context; new_session_context = Context::createCopy(global_context); @@ -364,6 +403,9 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: if (query_context_created) throw Exception("Session context must be created before any query context", ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(log, "{} Creating named session context with name: {}, user_id: {}", + toString(session_id), session_name_, user_id ? toString(*user_id) : ""); + /// Make a new session context OR /// if the `session_id` and `user_id` were used before then just get a previously created session context. std::shared_ptr new_named_session; @@ -420,6 +462,12 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t ContextMutablePtr query_context = Context::createCopy(from_session_context ? session_context : global_context); query_context->makeQueryContext(); + LOG_DEBUG(log, "{} Creating query context from {} context, user_id: {}, parent context user: {}", + toString(session_id), + from_session_context ? "session" : "global", + user_id ? toString(*user_id) : "", + query_context->getUser() ? query_context->getUser()->getName() : ""); + /// Copy the specified client info to the new query context. auto & res_client_info = query_context->getClientInfo(); if (client_info_to_move) @@ -460,7 +508,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (!notified_session_log_about_login) { - if (auto session_log = getSessionLog(); session_log && user) + if (auto session_log = getSessionLog(); user && user_id && session_log) { session_log->addLoginSuccess( session_id, diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 494b7750f1e..26457186523 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -90,6 +90,8 @@ private: std::shared_ptr named_session; bool named_session_created = false; + + Poco::Logger * log = nullptr; }; } From 905418b46868d1345d1de444003b5f057c616829 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 18:33:22 +0300 Subject: [PATCH 848/950] Update argmin.md because https://github.com/ClickHouse/ClickHouse/pull/23393 --- .../sql-reference/aggregate-functions/reference/argmin.md | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md index 4ee78a73a84..6205cd0cf09 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md @@ -14,12 +14,6 @@ toc_priority: 105 argMin(arg, val) ``` -или - -``` sql -argMin(tuple(arg, val)) -``` - **Аргументы** - `arg` — аргумент. From aa8bc93be8c2e245ebf12c7b5872910a2f071722 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 19 Oct 2021 23:35:59 +0800 Subject: [PATCH 849/950] Fix build --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index a61c2669ef2..106147d95fc 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1895,7 +1895,7 @@ void Context::setSystemZooKeeperLogAfterInitializationIfNeeded() zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log); } -void Context::initializeKeeperDispatcher(bool start_async) const +void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const { #if USE_NURAFT std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); From 05d93796dcf5ee30a9daebcd840e2b35fbf32fb2 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 19 Oct 2021 23:36:07 +0800 Subject: [PATCH 850/950] Fix minmax_count projection with primary key in partition expr --- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++++++---------- src/Storages/ProjectionsDescription.cpp | 20 ++++++++++++------- src/Storages/ProjectionsDescription.h | 4 ---- .../01710_minmax_count_projection.reference | 1 + .../01710_minmax_count_projection.sql | 6 ++++++ 5 files changed, 30 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 10fa18186ee..ced385a18c4 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4470,16 +4470,6 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } size_t pos = 0; - if (!primary_key_max_column_name.empty()) - { - const auto & primary_key_column = *part->index[0]; - auto primary_key_column_size = primary_key_column.size(); - auto & min_column = assert_cast(*minmax_count_columns[pos++]); - auto & max_column = assert_cast(*minmax_count_columns[pos++]); - insert(min_column, primary_key_column[0]); - insert(max_column, primary_key_column[primary_key_column_size - 1]); - } - size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); for (size_t i = 0; i < minmax_idx_size; ++i) { @@ -4490,6 +4480,16 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( insert(max_column, range.right); } + if (!primary_key_max_column_name.empty()) + { + const auto & primary_key_column = *part->index[0]; + auto primary_key_column_size = primary_key_column.size(); + auto & min_column = assert_cast(*minmax_count_columns[pos++]); + auto & max_column = assert_cast(*minmax_count_columns[pos++]); + insert(min_column, primary_key_column[0]); + insert(max_column, primary_key_column[primary_key_column_size - 1]); + } + { auto & column = assert_cast(*minmax_count_columns.back()); auto func = column.getAggregateFunction(); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index b204c288000..e5117a306ee 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -184,16 +184,16 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto select_query = std::make_shared(); ASTPtr select_expression_list = std::make_shared(); - if (!primary_key_asts.empty()) - { - select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); - select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); - } for (const auto & column : minmax_columns) { select_expression_list->children.push_back(makeASTFunction("min", std::make_shared(column))); select_expression_list->children.push_back(makeASTFunction("max", std::make_shared(column))); } + if (!primary_key_asts.empty()) + { + select_expression_list->children.push_back(makeASTFunction("min", primary_key_asts.front()->clone())); + select_expression_list->children.push_back(makeASTFunction("max", primary_key_asts.front()->clone())); + } select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); @@ -207,8 +207,14 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); - if (!primary_key_asts.empty()) - result.primary_key_max_column_name = result.sample_block.getNames()[ProjectionDescription::PRIMARY_KEY_MAX_COLUMN_POS]; + /// If we have primary key and it's not in minmax_columns, it will be used as one additional minmax columns. + if (!primary_key_asts.empty() && result.sample_block.columns() == 2 * (minmax_columns.size() + 1) + 1) + { + /// min(p1), max(p1), min(p2), max(p2), ..., min(k1), max(k1), count() + /// ^ + /// size - 2 + result.primary_key_max_column_name = *(result.sample_block.getNames().cend() - 2); + } result.type = ProjectionDescription::Type::Aggregate; StorageInMemoryMetadata metadata; metadata.setColumns(ColumnsDescription(result.sample_block.getNamesAndTypesList())); diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 4dd717239ad..7c254182ba4 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -30,10 +30,6 @@ struct ProjectionDescription static constexpr const char * MINMAX_COUNT_PROJECTION_NAME = "_minmax_count_projection"; - /// If minmax_count projection contains a primary key's minmax values. Their positions will be 0 and 1. - static constexpr const size_t PRIMARY_KEY_MIN_COLUMN_POS = 0; - static constexpr const size_t PRIMARY_KEY_MAX_COLUMN_POS = 1; - /// Definition AST of projection ASTPtr definition_ast; diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 5591d5a9954..da7d2fbe2bd 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -8,3 +8,4 @@ 0 0 9999 0 9999 +3 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 112487b219e..b7077de1fe6 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -43,3 +43,9 @@ select min(j), max(j) from has_final_mark; set max_rows_to_read = 5001; -- one normal part 5000 + one minmax_count_projection part 1 select min(j), max(j) from mixed_final_mark; + +-- The first primary expr is the same of some partition column +drop table if exists t; +create table t (server_date Date, something String) engine MergeTree partition by (toYYYYMM(server_date), server_date) order by (server_date, something); +insert into t values ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'); +select count() from t; From 0f2620a31ef37179c2009385835e490fc3baf89e Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:07:04 +0300 Subject: [PATCH 851/950] Update argmin.md --- .../aggregate-functions/reference/argmin.md | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md index 6205cd0cf09..4d2363abe6d 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md @@ -25,12 +25,6 @@ argMin(arg, val) Тип: соответствует типу `arg`. -Если передан кортеж: - -- Кортеж `(arg, val)` c минимальным значением `val` и соответствующим ему `arg`. - -Тип: [Tuple](../../../sql-reference/data-types/tuple.md). - **Пример** Исходная таблица: @@ -46,14 +40,14 @@ argMin(arg, val) Запрос: ``` sql -SELECT argMin(user, salary), argMin(tuple(user, salary)) FROM salary; +SELECT argMin(user, salary) FROM salary; ``` Результат: ``` text -┌─argMin(user, salary)─┬─argMin(tuple(user, salary))─┐ -│ worker │ ('worker',1000) │ -└──────────────────────┴─────────────────────────────┘ +┌─argMin(user, salary)─┐ +│ worker │ +└──────────────────────┘ ``` From c6af087f4bf89dd27f06cd6bb9972f6fc89e781b Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:08:50 +0300 Subject: [PATCH 852/950] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 20 ++++--------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md index 71289423035..84419523beb 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md @@ -14,12 +14,6 @@ toc_priority: 106 argMax(arg, val) ``` -или - -``` sql -argMax(tuple(arg, val)) -``` - **Аргументы** - `arg` — аргумент. @@ -31,12 +25,6 @@ argMax(tuple(arg, val)) Тип: соответствует типу `arg`. -Если передан кортеж: - -- кортеж `(arg, val)` c максимальным значением `val` и соответствующим ему `arg`. - -Тип: [Tuple](../../../sql-reference/data-types/tuple.md). - **Пример** Исходная таблица: @@ -52,14 +40,14 @@ argMax(tuple(arg, val)) Запрос: ``` sql -SELECT argMax(user, salary), argMax(tuple(user, salary), salary), argMax(tuple(user, salary)) FROM salary; +SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; ``` Результат: ``` text -┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┬─argMax(tuple(user, salary))─┐ -│ director │ ('director',5000) │ ('director',5000) │ -└──────────────────────┴─────────────────────────────────────┴─────────────────────────────┘ +┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┐ +│ director │ ('director',5000) │ +└──────────────────────┴─────────────────────────────────────┘ ``` From f49e946943fe6bd6d9da9d0b1a99539bd5eac37a Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:11:15 +0300 Subject: [PATCH 853/950] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 20 ++++--------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md index 9d90590b2f1..0c82cb8a4bb 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md @@ -14,12 +14,6 @@ toc_priority: 106 argMax(arg, val) ``` -或 - -``` sql -argMax(tuple(arg, val)) -``` - **参数** - `arg` — Argument. @@ -31,12 +25,6 @@ argMax(tuple(arg, val)) 类型: 匹配 `arg` 类型。 -对于输入中的元组: - -- 元组 `(arg, val)`, 其中 `val` 最大值,`arg` 是对应的值。 - -类型: [元组](../../../sql-reference/data-types/tuple.md)。 - **示例** 输入表: @@ -52,13 +40,13 @@ argMax(tuple(arg, val)) 查询: ``` sql -SELECT argMax(user, salary), argMax(tuple(user, salary), salary), argMax(tuple(user, salary)) FROM salary; +SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; ``` 结果: ``` text -┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┬─argMax(tuple(user, salary))─┐ -│ director │ ('director',5000) │ ('director',5000) │ -└──────────────────────┴─────────────────────────────────────┴─────────────────────────────┘ +┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┐ +│ director │ ('director',5000) │ +└──────────────────────┴─────────────────────────────────────┘ ``` From 2b3841f0030c0a702ea88ef6fe4499f9ad6d0705 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:14:14 +0300 Subject: [PATCH 854/950] Update argmin.md --- .../sql-reference/aggregate-functions/reference/argmin.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md index 0dd4625ac0d..6c2839062eb 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md @@ -25,13 +25,13 @@ toc_priority: 105 查询: ``` sql -SELECT argMin(user, salary), argMin(tuple(user, salary)) FROM salary; +SELECT argMin(user, salary), argMin(tuple(user, salary), salary) FROM salary; ``` 结果: ``` text -┌─argMin(user, salary)─┬─argMin(tuple(user, salary))─┐ -│ worker │ ('worker',1000) │ -└──────────────────────┴─────────────────────────────┘ +┌─argMin(user, salary)─┬─argMin(tuple(user, salary), salary)─┐ +│ worker │ ('worker',1000) │ +└──────────────────────┴─────────────────────────────────────┘ ``` From e72ec27d5bfc4f6056ad88f775bc234b921e3d9e Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 19 Oct 2021 19:45:48 +0300 Subject: [PATCH 855/950] Fixed builds and using magic_enum --- src/Interpreters/Session.cpp | 30 ++++-------------------------- 1 file changed, 4 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 75117bd860c..bb3d9352496 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -4,13 +4,15 @@ #include #include #include -#include +#include #include #include #include #include #include +#include + #include #include #include @@ -241,34 +243,10 @@ void Session::shutdownNamedSessions() NamedSessionsStorage::instance().shutdown(); } -namespace -{ -String getSessionPrefix(ClientInfo::Interface interface) -{ - switch (interface) - { - case ClientInfo::Interface::TCP: - return "TCP"; - case ClientInfo::Interface::HTTP: - return "HTTP"; - case ClientInfo::Interface::GRPC: - return "GRPC"; - case ClientInfo::Interface::MYSQL: - return "MySQL"; - case ClientInfo::Interface::POSTGRESQL: - return "PostgreSQL"; - case ClientInfo::Interface::LOCAL: - return "Local"; - case ClientInfo::Interface::TCP_INTERSERVER: - return "Interserver"; - } -} -} - Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_) : session_id(UUIDHelpers::generateV4()), global_context(global_context_), - log(&Poco::Logger::get(getSessionPrefix(interface_) + "-Session")) + log(&Poco::Logger::get(String{magic_enum::enum_name(interface_)} + "-Session")) { prepared_client_info.emplace(); prepared_client_info->interface = interface_; From 377f6cf3c251bfc27f2f3c2ccf162c0ef8dab903 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 19 Oct 2021 21:51:46 +0300 Subject: [PATCH 856/950] Edit and translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../external-dicts-dict-layout.md | 14 +++-- .../external-dicts-dict-layout.md | 60 ++++++++++++++++++- 2 files changed, 69 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index c4a359efca4..766ce9a7a7e 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -52,6 +52,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) +- [hashed_array](#dicts-external_dicts_dict_layout-hashed-array) - [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [ssd_cache](#ssd-cache) @@ -151,13 +152,15 @@ Configuration example: ``` +or + ``` sql LAYOUT(COMPLEX_KEY_HASHED()) ``` ### complex_key_sparse_hashed {#complex-key-sparse-hashed} -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `sparse_hashed`. +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed). Configuration example: @@ -167,13 +170,15 @@ Configuration example: ``` +or + ``` sql LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) ``` ### hashed_array {#dicts-external_dicts_dict_layout-hashed-array} -The dictionary is completely stored in memory. Each attribute is stored in array. Key attribute is stored in the form of hashed table where value is index in attributes array. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. +The dictionary is completely stored in memory. Each attribute is stored in an array. The key attribute is stored in the form of a hashed table where value is an index in the attributes array. The dictionary can contain any number of elements with any identifiers. In practice, the number of keys can reach tens of millions of items. All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. @@ -194,7 +199,7 @@ LAYOUT(HASHED_ARRAY()) ### complex_key_hashed_array {#complex-key-hashed-array} -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed_array`. +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to [hashed_array](#dicts-external_dicts_dict_layout-hashed-array). Configuration example: @@ -204,11 +209,12 @@ Configuration example: ``` +or + ``` sql LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) ``` - ### range_hashed {#range-hashed} The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 73736344c57..82937a90ee5 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -52,6 +52,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) +- [hashed_array](#dicts-external_dicts_dict_layout-hashed-array) - [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [ssd_cache](#ssd-cache) @@ -140,7 +141,7 @@ LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) ### complex_key_hashed {#complex-key-hashed} -Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `hashed`. +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Аналогичен `hashed`. Пример конфигурации: @@ -156,6 +157,63 @@ LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) LAYOUT(COMPLEX_KEY_HASHED()) ``` +### complex_key_sparse_hashed {#complex-key-sparse-hashed} + +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Аналогичен [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed). + +Пример конфигурации: + +``` xml + + + +``` + +или + +``` sql +LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) +``` + +### hashed_array {#dicts-external_dicts_dict_layout-hashed-array} + +Словарь полностью хранится в оперативной памяти. Каждый атрибут хранится в массиве. Ключевой атрибут хранится в виде хэш-таблицы, где его значение является индексом в массиве атрибутов. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике количество ключей может достигать десятков миллионов элементов. + +Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком. + +Пример конфигурации: + +``` xml + + + + +``` + +или + +``` sql +LAYOUT(HASHED_ARRAY()) +``` + +### complex_key_hashed_array {#complex-key-hashed-array} + +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Аналогичен [hashed_array](#dicts-external_dicts_dict_layout-hashed-array). + +Пример конфигурации: + +``` xml + + + +``` + +или + +``` sql +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) +``` + ### range_hashed {#range-hashed} Словарь хранится в оперативной памяти в виде хэш-таблицы с упорядоченным массивом диапазонов и соответствующих им значений. From b7fdf488e16b9f5023358bd22932a8b000be4abb Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 19 Oct 2021 22:38:00 +0300 Subject: [PATCH 857/950] Translate install.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Перевел на русский язык предложение в install.md. --- docs/en/getting-started/install.md | 3 +-- docs/ru/getting-started/install.md | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 06186842809..4a97ab6589d 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -27,11 +27,10 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun {% include 'install/deb.sh' %} ``` -You can replace `stable` with `lts` or `testing` to use different [“release trains”](../faq/operations/production.md) based on your needs. +You can replace `stable` with `lts` or `testing` to use different [release trains](../faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/). - #### Packages {#packages} - `clickhouse-common-static` — Installs ClickHouse compiled binary files. diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 5cb75c12155..a12773a75b0 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -29,7 +29,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su Также эти пакеты можно скачать и установить вручную отсюда: https://repo.clickhouse.com/deb/stable/main/. -Если вы хотите использовать наиболее свежую версию, замените `stable` на `testing` (рекомендуется для тестовых окружений). +Чтобы использовать различные [версии ClickHouse](../faq/operations/production.md) в зависимости от ваших потребностей, вы можете заменить `stable` на `lts` или `testing`. Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.com/deb/stable/main/). From 9699a71806c63f6fd47099bde51031bb12f403c9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 22:39:55 +0300 Subject: [PATCH 858/950] Update amis --- tests/ci/metrics_lambda/app.py | 2 +- tests/ci/termination_lambda/app.py | 2 +- tests/ci/token_lambda/app.py | 2 +- tests/ci/worker/ubuntu_ami.sh | 47 ++++++++++++++++++++++++++++++ 4 files changed, 50 insertions(+), 3 deletions(-) create mode 100644 tests/ci/worker/ubuntu_ami.sh diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index 6c6fc594847..d2fb048638b 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -10,7 +10,7 @@ from collections import namedtuple def get_key_and_app_from_aws(): import boto3 - secret_name = "clickhouse_github_secret_key_1" + secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( service_name='secretsmanager', diff --git a/tests/ci/termination_lambda/app.py b/tests/ci/termination_lambda/app.py index 7fd7c400db9..0b39cf73f25 100644 --- a/tests/ci/termination_lambda/app.py +++ b/tests/ci/termination_lambda/app.py @@ -10,7 +10,7 @@ from collections import namedtuple def get_key_and_app_from_aws(): import boto3 - secret_name = "clickhouse_github_secret_key_1" + secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( service_name='secretsmanager', diff --git a/tests/ci/token_lambda/app.py b/tests/ci/token_lambda/app.py index 4edd3e8d08c..731d6c040de 100644 --- a/tests/ci/token_lambda/app.py +++ b/tests/ci/token_lambda/app.py @@ -39,7 +39,7 @@ def get_runner_registration_token(access_token): def get_key_and_app_from_aws(): import boto3 - secret_name = "clickhouse_github_secret_key_1" + secret_name = "clickhouse_github_secret_key" session = boto3.session.Session() client = session.client( service_name='secretsmanager', diff --git a/tests/ci/worker/ubuntu_ami.sh b/tests/ci/worker/ubuntu_ami.sh new file mode 100644 index 00000000000..2609c1a69f3 --- /dev/null +++ b/tests/ci/worker/ubuntu_ami.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash +set -euo pipefail + +echo "Running prepare script" +export DEBIAN_FRONTEND=noninteractive +export RUNNER_VERSION=2.283.1 +export RUNNER_HOME=/home/ubuntu/actions-runner + +apt-get update + +apt-get install --yes --no-install-recommends \ + apt-transport-https \ + ca-certificates \ + curl \ + gnupg \ + lsb-release \ + python3-pip \ + unzip + +curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg + +echo "deb [arch=amd64 signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null + +apt-get update + +apt-get install --yes --no-install-recommends docker-ce docker-ce-cli containerd.io + +usermod -aG docker ubuntu + +pip install boto3 pygithub requests urllib3 unidiff + +mkdir -p $RUNNER_HOME && cd $RUNNER_HOME + +curl -O -L https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/actions-runner-linux-x64-$RUNNER_VERSION.tar.gz + +tar xzf ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz +rm -f ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz +./bin/installdependencies.sh + +chown -R ubuntu:ubuntu $RUNNER_HOME + +cd /home/ubuntu +curl "https://awscli.amazonaws.com/awscli-exe-linux-x86_64.zip" -o "awscliv2.zip" +unzip awscliv2.zip +./aws/install + +rm -rf /home/ubuntu/awscliv2.zip /home/ubuntu/aws From 2266856c472cf083e598af4dad25491cfd8927ca Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 19 Oct 2021 23:03:45 +0300 Subject: [PATCH 859/950] Follow-up for #30282 --- src/Interpreters/Context.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 243bc721f92..84795f11f2a 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -293,8 +293,8 @@ private: /// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL). bool is_internal_query = false; - /// Has initializeBackgroundExecutors() method been executed? - bool is_background_executors_initialized = false; + /// Has background executors for MergeTree tables been initialized? + std::atomic is_background_executors_initialized = false; public: From 008a13df8b7398cd4c6ebbb4530f791a9f2cc2c6 Mon Sep 17 00:00:00 2001 From: lhuang0928 Date: Wed, 20 Oct 2021 03:36:07 +0000 Subject: [PATCH 860/950] fix date32 comparison with datetime/datetime64 --- src/Functions/FunctionsConversion.h | 4 ++-- src/Interpreters/convertFieldToType.cpp | 2 +- .../0_stateless/02098_date32_comparison.reference | 11 +++++++++++ .../queries/0_stateless/02098_date32_comparison.sql | 13 ++++++++++++- 4 files changed, 26 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 95db7a9af25..9238cc81c37 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -301,7 +301,7 @@ struct ToDateTimeImpl return time_zone.fromDayNum(DayNum(d)); } - static inline UInt32 execute(Int32 d, const DateLUTImpl & time_zone) + static inline Int64 execute(Int32 d, const DateLUTImpl & time_zone) { return time_zone.fromDayNum(ExtendedDayNum(d)); } @@ -638,7 +638,7 @@ struct ToDateTime64Transform inline DateTime64::NativeType execute(Int32 d, const DateLUTImpl & time_zone) const { const auto dt = ToDateTimeImpl::execute(d, time_zone); - return execute(dt, time_zone); + return DecimalUtils::decimalFromComponentsWithMultiplier(dt, 0, scale_multiplier); } inline DateTime64::NativeType execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) const diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index ef86f45b759..a91cb1fbeb2 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -216,7 +216,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } if (which_type.isDateTime64() - && (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDateTime() || which_from_type.isDateTime64())) + && (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDate32() || which_from_type.isDateTime() || which_from_type.isDateTime64())) { const auto scale = static_cast(type).getScale(); const auto decimal_value = DecimalUtils::decimalFromComponents(src.reinterpret(), 0, scale); diff --git a/tests/queries/0_stateless/02098_date32_comparison.reference b/tests/queries/0_stateless/02098_date32_comparison.reference index c18b4e9b082..16d1f96acfd 100644 --- a/tests/queries/0_stateless/02098_date32_comparison.reference +++ b/tests/queries/0_stateless/02098_date32_comparison.reference @@ -6,3 +6,14 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02098_date32_comparison.sql b/tests/queries/0_stateless/02098_date32_comparison.sql index dd4fde790c7..5fd7172e0bb 100644 --- a/tests/queries/0_stateless/02098_date32_comparison.sql +++ b/tests/queries/0_stateless/02098_date32_comparison.sql @@ -1,8 +1,19 @@ select toDate32('1990-01-01') = toDate('1990-01-01'); select toDate('1991-01-02') > toDate32('1990-01-01'); +select toDate32('1925-01-01') <= toDate('1990-01-01'); +select toDate('1991-01-01') < toDate32('2283-11-11'); select toDate32('1990-01-01') = toDateTime('1990-01-01'); select toDateTime('1991-01-02') > toDate32('1990-01-01'); +select toDate32('1925-01-01') <= toDateTime('1990-01-01'); +select toDateTime('1991-01-01') < toDate32('2283-11-11'); select toDate32('1990-01-01') = toDateTime64('1990-01-01',2); select toDateTime64('1991-01-02',2) > toDate32('1990-01-01'); +select toDate32('1925-01-01') = toDateTime64('1925-01-01',2); +select toDateTime64('1925-01-02',2) > toDate32('1925-01-01'); +select toDate32('2283-11-11') = toDateTime64('2283-11-11',2); +select toDateTime64('2283-11-11',2) > toDate32('1925-01-01'); select toDate32('1990-01-01') = '1990-01-01'; -select '1991-01-02' > toDate32('1990-01-01'); \ No newline at end of file +select '1991-01-02' > toDate32('1990-01-01'); +select toDate32('1925-01-01') = '1925-01-01'; +select '2283-11-11' >= toDate32('2283-11-10'); +select '2283-11-11' > toDate32('1925-01-01'); \ No newline at end of file From 8f97765cbdec3c7ed9c37df46c18ef131ac92f00 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Oct 2021 21:06:08 +0000 Subject: [PATCH 861/950] Fix --- src/Interpreters/Context.h | 2 +- src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 84795f11f2a..85b4fbf23d5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -294,7 +294,7 @@ private: /// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL). bool is_internal_query = false; /// Has background executors for MergeTree tables been initialized? - std::atomic is_background_executors_initialized = false; + bool is_background_executors_initialized = false; public: diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 982acfe62a4..218d990e0ce 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -301,6 +301,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) name_part = name_part.substr(strlen("Replicated")); + args.getContext()->getGlobalContext()->initializeBackgroundExecutorsIfNeeded(); + MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; From 5e24f337a049f2aaee9efc8dc03d354103ab0ca2 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 20 Oct 2021 09:29:41 +0300 Subject: [PATCH 862/950] Update StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f554986402c..ba115d36a5d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5995,12 +5995,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom( MutableDataPartsVector dst_parts; Strings block_id_paths; Strings part_checksums; + auto zookeeper = getZooKeeper(); std::vector ephemeral_locks; LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); static const String TMP_PREFIX = "tmp_replace_from_"; - auto zookeeper = getZooKeeper(); String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; Coordination::Stat alter_partition_version_stat; From 36635736e6aaaf60c1faf7e1632384e06a278313 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 12:30:17 +0300 Subject: [PATCH 863/950] Also run on master --- .github/workflows/main.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 49760995dfc..baa40e99418 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -1,5 +1,8 @@ name: Ligthweight GithubActions on: # yamllint disable-line rule:truthy + push: + branches: + - master pull_request: types: - labeled From 83787e26f2f1a287c12886ad316733dcbc3676cf Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 12:43:03 +0300 Subject: [PATCH 864/950] SQLUserDefinedFunctions support lambdas --- .../InterpreterCreateFunctionQuery.cpp | 40 +++---------------- .../InterpreterCreateFunctionQuery.h | 1 - .../UserDefinedSQLFunctionVisitor.cpp | 10 ++++- ...l_user_defined_functions_aliases.reference | 1 + ...098_sql_user_defined_functions_aliases.sql | 4 ++ ...ql_user_defined_functions_lambda.reference | 1 + ...2099_sql_user_defined_functions_lambda.sql | 4 ++ 7 files changed, 24 insertions(+), 37 deletions(-) create mode 100644 tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference create mode 100644 tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql create mode 100644 tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.reference create mode 100644 tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index ccb5f4040ec..9d92466c440 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -1,14 +1,17 @@ +#include + +#include + #include #include #include #include #include #include -#include #include #include #include -#include + namespace DB { @@ -66,42 +69,9 @@ void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function, const Str } ASTPtr function_body = function->as()->children.at(0)->children.at(1); - std::unordered_set identifiers_in_body = getIdentifiers(function_body); - - for (const auto & identifier : identifiers_in_body) - { - if (!arguments.contains(identifier)) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Identifier {} does not exist in arguments", backQuote(identifier)); - } - validateFunctionRecursiveness(function_body, name); } -std::unordered_set InterpreterCreateFunctionQuery::getIdentifiers(ASTPtr node) -{ - std::unordered_set identifiers; - - std::stack ast_nodes_to_process; - ast_nodes_to_process.push(node); - - while (!ast_nodes_to_process.empty()) - { - auto ast_node_to_process = ast_nodes_to_process.top(); - ast_nodes_to_process.pop(); - - for (const auto & child : ast_node_to_process->children) - { - auto identifier_name_opt = tryGetIdentifierName(child); - if (identifier_name_opt) - identifiers.insert(identifier_name_opt.value()); - - ast_nodes_to_process.push(child); - } - } - - return identifiers; -} - void InterpreterCreateFunctionQuery::validateFunctionRecursiveness(ASTPtr node, const String & function_to_create) { for (const auto & child : node->children) diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.h b/src/Interpreters/InterpreterCreateFunctionQuery.h index fdc03b379db..a67fdb9605d 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.h +++ b/src/Interpreters/InterpreterCreateFunctionQuery.h @@ -22,7 +22,6 @@ public: private: static void validateFunction(ASTPtr function, const String & name); - static std::unordered_set getIdentifiers(ASTPtr node); static void validateFunctionRecursiveness(ASTPtr node, const String & function_to_create); ASTPtr query_ptr; diff --git a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp index cc5db020387..8df6932b153 100644 --- a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp @@ -25,6 +25,7 @@ void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data &) return; auto result = tryToReplaceFunction(*function); + if (result) ast = result; } @@ -83,9 +84,16 @@ ASTPtr UserDefinedSQLFunctionMatcher::tryToReplaceFunction(const ASTFunction & f if (identifier_name_opt) { auto function_argument_it = identifier_name_to_function_argument.find(*identifier_name_opt); - assert(function_argument_it != identifier_name_to_function_argument.end()); + if (function_argument_it == identifier_name_to_function_argument.end()) + continue; + + auto child_alias = child->tryGetAlias(); child = function_argument_it->second->clone(); + + if (!child_alias.empty()) + child->setAlias(child_alias); + continue; } diff --git a/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference new file mode 100644 index 00000000000..8ab2f6d0ac6 --- /dev/null +++ b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference @@ -0,0 +1 @@ +8 4 diff --git a/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql new file mode 100644 index 00000000000..fef2daf8fd1 --- /dev/null +++ b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql @@ -0,0 +1,4 @@ +-- Tags: no-parallel +CREATE FUNCTION alias_function AS x -> (((x * 2) AS x_doubled) + x_doubled); +SELECT alias_function(2); +DROP FUNCTION alias_function; diff --git a/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.reference b/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.reference new file mode 100644 index 00000000000..8f6cd5ccd03 --- /dev/null +++ b/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.reference @@ -0,0 +1 @@ +[2,4,6] diff --git a/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql b/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql new file mode 100644 index 00000000000..e66651c1e5a --- /dev/null +++ b/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql @@ -0,0 +1,4 @@ +-- Tags: no-parallel +CREATE FUNCTION lambda_function AS x -> arrayMap(array_element -> array_element * 2, x); +SELECT lambda_function([1,2,3]); +DROP FUNCTION lambda_function; From 75d77339e9b05f15a4de59cf76086ec59838149e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 12:45:02 +0300 Subject: [PATCH 865/950] Remove master --- .github/workflows/main.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index baa40e99418..49760995dfc 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -1,8 +1,5 @@ name: Ligthweight GithubActions on: # yamllint disable-line rule:truthy - push: - branches: - - master pull_request: types: - labeled From 0ad7f9bba25e4cbf53dfea3b1fbd51379aab2dc3 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 20 Oct 2021 12:45:32 +0300 Subject: [PATCH 866/950] Fixed PVS warning --- src/Interpreters/Session.cpp | 2 -- src/Interpreters/Session.h | 3 ++- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index bb3d9352496..865c9551219 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -252,8 +252,6 @@ Session::Session(const ContextPtr & global_context_, ClientInfo::Interface inter prepared_client_info->interface = interface_; } -Session::Session(Session &&) = default; - Session::~Session() { LOG_DEBUG(log, "{} Destroying {} of user {}", diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 26457186523..b62327103e9 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -33,9 +33,10 @@ public: static void shutdownNamedSessions(); Session(const ContextPtr & global_context_, ClientInfo::Interface interface_); - Session(Session &&); ~Session(); + Session(const Session &&) = delete; + Session& operator=(const Session &&) = delete; Session(const Session &) = delete; Session& operator=(const Session &) = delete; From 3496cd1bfec9ac4bce9119651fbb14f58e1824d3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 12:53:03 +0300 Subject: [PATCH 867/950] ExecutableUDF example --- src/Common/examples/CMakeLists.txt | 3 ++ src/Common/examples/executable_udf.cpp | 44 ++++++++++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 src/Common/examples/executable_udf.cpp diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index e72681621cb..020f3cc4446 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -77,3 +77,6 @@ target_link_libraries (average PRIVATE clickhouse_common_io) add_executable (shell_command_inout shell_command_inout.cpp) target_link_libraries (shell_command_inout PRIVATE clickhouse_common_io) + +add_executable (executable_udf executable_udf.cpp) +target_link_libraries (executable_udf PRIVATE dbms) diff --git a/src/Common/examples/executable_udf.cpp b/src/Common/examples/executable_udf.cpp new file mode 100644 index 00000000000..78a248fcddf --- /dev/null +++ b/src/Common/examples/executable_udf.cpp @@ -0,0 +1,44 @@ +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +using namespace DB; + +int main(int argc, char **argv) +{ + (void)(argc); + (void)(argv); + + std::string buffer; + + ReadBufferFromFileDescriptor read_buffer(0); + WriteBufferFromFileDescriptor write_buffer(1); + size_t rows = 0; + char dummy; + + while (!read_buffer.eof()) { + readIntText(rows, read_buffer); + readChar(dummy, read_buffer); + + for (size_t i = 0; i < rows; ++i) { + readString(buffer, read_buffer); + readChar(dummy, read_buffer); + + writeString("Key ", write_buffer); + writeString(buffer, write_buffer); + writeChar('\n', write_buffer); + } + + write_buffer.next(); + } + + return 0; +} From ddcf12c83b94f4e1c08d74289ce50b6300a70da4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 20 Oct 2021 11:52:48 +0800 Subject: [PATCH 868/950] JSONExtractString raw string support. --- src/Functions/FunctionsJSON.cpp | 17 +++++++---------- .../01915_json_extract_raw_string.reference | 3 +++ .../01915_json_extract_raw_string.sql | 4 ++++ 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index cb55ba6b83b..e861e99861b 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -744,6 +744,8 @@ public: } }; +template +class JSONExtractRawImpl; template class JSONExtractStringImpl @@ -760,9 +762,12 @@ public: static bool insertResultToColumn(IColumn & dest, const Element & element, const std::string_view &) { - if (!element.isString()) + if (element.isNull()) return false; + if (!element.isString()) + return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); + auto str = element.getString(); ColumnString & col_str = assert_cast(dest); col_str.insertData(str.data(), str.size()); @@ -770,9 +775,6 @@ public: } }; -template -class JSONExtractRawImpl; - /// Nodes of the extract tree. We need the extract tree to extract from JSON complex values containing array, tuples or nullables. template struct JSONExtractTree @@ -856,12 +858,7 @@ struct JSONExtractTree public: bool insertResultToColumn(IColumn & dest, const Element & element) override { - if (element.isString()) - return JSONExtractStringImpl::insertResultToColumn(dest, element, {}); - else if (element.isNull()) - return false; - else - return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); + return JSONExtractStringImpl::insertResultToColumn(dest, element, {}); } }; diff --git a/tests/queries/0_stateless/01915_json_extract_raw_string.reference b/tests/queries/0_stateless/01915_json_extract_raw_string.reference index 3a41f35710c..e88c7e018d2 100644 --- a/tests/queries/0_stateless/01915_json_extract_raw_string.reference +++ b/tests/queries/0_stateless/01915_json_extract_raw_string.reference @@ -1,2 +1,5 @@ ('123','456','[7,8,9]') \N +123 +123 + diff --git a/tests/queries/0_stateless/01915_json_extract_raw_string.sql b/tests/queries/0_stateless/01915_json_extract_raw_string.sql index 884c599c206..98bff692d71 100644 --- a/tests/queries/0_stateless/01915_json_extract_raw_string.sql +++ b/tests/queries/0_stateless/01915_json_extract_raw_string.sql @@ -3,3 +3,7 @@ select JSONExtract('{"a": "123", "b": 456, "c": [7, 8, 9]}', 'Tuple(a String, b String, c String)'); with '{"string_value":null}' as json select JSONExtract(json, 'string_value', 'Nullable(String)'); + +select JSONExtractString('{"a": 123}', 'a'); +select JSONExtractString('{"a": "123"}', 'a'); +select JSONExtractString('{"a": null}', 'a'); From ad409d9b47ae1dd3c492a46e32d0f66701dd122c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 13:05:57 +0300 Subject: [PATCH 869/950] SQLUserDefinedFunctions added DROP IF EXISTS support --- src/Interpreters/InterpreterDropFunctionQuery.cpp | 5 +++++ src/Interpreters/UserDefinedSQLFunctionFactory.cpp | 5 +++++ src/Interpreters/UserDefinedSQLFunctionFactory.h | 2 ++ src/Parsers/ASTDropFunctionQuery.cpp | 2 ++ src/Parsers/ASTDropFunctionQuery.h | 2 ++ src/Parsers/ParserDropFunctionQuery.cpp | 7 +++++++ ...1_sql_user_defined_functions_drop_if_exists.reference | 1 + .../02101_sql_user_defined_functions_drop_if_exists.sql | 9 +++++++++ 8 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.reference create mode 100644 tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.sql diff --git a/src/Interpreters/InterpreterDropFunctionQuery.cpp b/src/Interpreters/InterpreterDropFunctionQuery.cpp index 53cb96b42fe..b788c8f960f 100644 --- a/src/Interpreters/InterpreterDropFunctionQuery.cpp +++ b/src/Interpreters/InterpreterDropFunctionQuery.cpp @@ -18,6 +18,11 @@ BlockIO InterpreterDropFunctionQuery::execute() FunctionNameNormalizer().visit(query_ptr.get()); auto & drop_function_query = query_ptr->as(); + auto & user_defined_functions_factory = UserDefinedSQLFunctionFactory::instance(); + + if (drop_function_query.if_exists && !user_defined_functions_factory.has(drop_function_query.function_name)) + return {}; + UserDefinedSQLFunctionFactory::instance().unregisterFunction(drop_function_query.function_name); UserDefinedSQLObjectsLoader::instance().removeObject(current_context, UserDefinedSQLObjectType::Function, drop_function_query.function_name); diff --git a/src/Interpreters/UserDefinedSQLFunctionFactory.cpp b/src/Interpreters/UserDefinedSQLFunctionFactory.cpp index 434f5523b42..1d2a80305c6 100644 --- a/src/Interpreters/UserDefinedSQLFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedSQLFunctionFactory.cpp @@ -77,6 +77,11 @@ ASTPtr UserDefinedSQLFunctionFactory::tryGet(const std::string & function_name) return it->second; } +bool UserDefinedSQLFunctionFactory::has(const String & function_name) const +{ + return tryGet(function_name) != nullptr; +} + std::vector UserDefinedSQLFunctionFactory::getAllRegisteredNames() const { std::vector registered_names; diff --git a/src/Interpreters/UserDefinedSQLFunctionFactory.h b/src/Interpreters/UserDefinedSQLFunctionFactory.h index 366e27e833d..6838c2f9892 100644 --- a/src/Interpreters/UserDefinedSQLFunctionFactory.h +++ b/src/Interpreters/UserDefinedSQLFunctionFactory.h @@ -23,6 +23,8 @@ public: ASTPtr tryGet(const String & function_name) const; + bool has(const String & function_name) const; + std::vector getAllRegisteredNames() const override; private: diff --git a/src/Parsers/ASTDropFunctionQuery.cpp b/src/Parsers/ASTDropFunctionQuery.cpp index 5800a7ba9cb..0a46940e73d 100644 --- a/src/Parsers/ASTDropFunctionQuery.cpp +++ b/src/Parsers/ASTDropFunctionQuery.cpp @@ -14,6 +14,8 @@ void ASTDropFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAS { settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP FUNCTION " << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : ""); + if (if_exists) + settings.ostr << "IF EXISTS"; } } diff --git a/src/Parsers/ASTDropFunctionQuery.h b/src/Parsers/ASTDropFunctionQuery.h index e32bf93a64d..a9d70a3016f 100644 --- a/src/Parsers/ASTDropFunctionQuery.h +++ b/src/Parsers/ASTDropFunctionQuery.h @@ -10,6 +10,8 @@ class ASTDropFunctionQuery : public IAST public: String function_name; + bool if_exists = false; + String getID(char) const override { return "DropFunctionQuery"; } ASTPtr clone() const override; diff --git a/src/Parsers/ParserDropFunctionQuery.cpp b/src/Parsers/ParserDropFunctionQuery.cpp index 04d26109836..d8c86646410 100644 --- a/src/Parsers/ParserDropFunctionQuery.cpp +++ b/src/Parsers/ParserDropFunctionQuery.cpp @@ -11,7 +11,10 @@ bool ParserDropFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expec { ParserKeyword s_drop("DROP"); ParserKeyword s_function("FUNCTION"); + ParserKeyword s_if_exists("IF EXISTS"); + ParserIdentifier function_name_p; + bool if_exists = false; ASTPtr function_name; @@ -21,10 +24,14 @@ bool ParserDropFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expec if (!s_function.ignore(pos, expected)) return false; + if (s_if_exists.ignore(pos, expected)) + if_exists = true; + if (!function_name_p.parse(pos, function_name, expected)) return false; auto drop_function_query = std::make_shared(); + drop_function_query->if_exists = if_exists; node = drop_function_query; drop_function_query->function_name = function_name->as().name(); diff --git a/tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.reference b/tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.sql b/tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.sql new file mode 100644 index 00000000000..09e2677774c --- /dev/null +++ b/tests/queries/0_stateless/02101_sql_user_defined_functions_drop_if_exists.sql @@ -0,0 +1,9 @@ +-- Tags: no-parallel + +CREATE FUNCTION 02101_test_function AS x -> x + 1; + +SELECT 02101_test_function(1); + +DROP FUNCTION 02101_test_function; +DROP FUNCTION 02101_test_function; --{serverError 46} +DROP FUNCTION IF EXISTS 02101_test_function; From fe93533ba45727225ff2e00cfcb87e3bc753d813 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 13:09:33 +0300 Subject: [PATCH 870/950] Fixed tests --- .../02098_sql_user_defined_functions_aliases.sql | 6 +++--- .../0_stateless/02099_sql_user_defined_functions_lambda.sql | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql index fef2daf8fd1..c5bd2b5b5f2 100644 --- a/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql +++ b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.sql @@ -1,4 +1,4 @@ -- Tags: no-parallel -CREATE FUNCTION alias_function AS x -> (((x * 2) AS x_doubled) + x_doubled); -SELECT alias_function(2); -DROP FUNCTION alias_function; +CREATE FUNCTION 02098_alias_function AS x -> (((x * 2) AS x_doubled) + x_doubled); +SELECT 02098_alias_function(2); +DROP FUNCTION 02098_alias_function; diff --git a/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql b/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql index e66651c1e5a..1c926faf3a1 100644 --- a/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql +++ b/tests/queries/0_stateless/02099_sql_user_defined_functions_lambda.sql @@ -1,4 +1,4 @@ -- Tags: no-parallel -CREATE FUNCTION lambda_function AS x -> arrayMap(array_element -> array_element * 2, x); -SELECT lambda_function([1,2,3]); -DROP FUNCTION lambda_function; +CREATE FUNCTION 02099_lambda_function AS x -> arrayMap(array_element -> array_element * 2, x); +SELECT 02099_lambda_function([1,2,3]); +DROP FUNCTION 02099_lambda_function; From dc964080126b5446ff9dec209ce9f8a7fa2a648e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 13:09:48 +0300 Subject: [PATCH 871/950] Fixed build --- src/Interpreters/InterpreterCreateFunctionQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 9d92466c440..c3d02fa4f34 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_IDENTIFIER; extern const int CANNOT_CREATE_RECURSIVE_FUNCTION; extern const int UNSUPPORTED_METHOD; } From ba442b7ce5e04775801f2e7118eb05111e6cc200 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 13:31:48 +0300 Subject: [PATCH 872/950] Use robot token in actions for statuses --- .github/workflows/main.yml | 20 ++++++++++++++++---- tests/ci/docker_images_check.py | 3 ++- tests/ci/finish_check.py | 3 ++- tests/ci/get_robot_token.py | 14 ++++++++++++++ tests/ci/pvs_check.py | 3 ++- tests/ci/run_check.py | 3 ++- tests/ci/style_check.py | 3 ++- 7 files changed, 40 insertions(+), 9 deletions(-) create mode 100644 tests/ci/get_robot_token.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 49760995dfc..60be4368df7 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -18,7 +18,10 @@ jobs: - name: Labels check run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DockerHubPush: needs: CheckLabels runs-on: [self-hosted] @@ -30,7 +33,10 @@ jobs: env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} - name: Upload images files to artifacts uses: actions/upload-artifact@v2 @@ -52,7 +58,10 @@ jobs: env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels] @@ -63,4 +72,7 @@ jobs: - name: Finish label run: cd $GITHUB_WORKSPACE/tests/ci && python3 finish_check.py env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 141d075cc6d..d1954d70e71 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,6 +8,7 @@ import os from pr_info import PRInfo from github import Github import shutil +from get_robot_token import get_best_robot_token NAME = "Push to Dockerhub (actions)" @@ -222,7 +223,7 @@ if __name__ == "__main__": url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) - gh = Github(os.getenv("GITHUB_TOKEN")) + gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=status, target_url=url) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 89139468fd6..db405cf8f73 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -4,6 +4,7 @@ from github import Github from pr_info import PRInfo import json import os +from get_robot_token import get_best_robot_token NAME = 'Run Check (actions)' @@ -34,7 +35,7 @@ if __name__ == "__main__": event = json.load(event_file) pr_info = PRInfo(event, need_orgs=True) - gh = Github(os.getenv("GITHUB_TOKEN")) + gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py new file mode 100644 index 00000000000..75b688e5b44 --- /dev/null +++ b/tests/ci/get_robot_token.py @@ -0,0 +1,14 @@ +#!/usr/bin/env python3 +import os +from github import Github + +def get_best_robot_token(token_prefix_env_name="ROBOT_TOKEN_", total_tokens=4): + tokens = {} + for i in range(total_tokens): + token_name = token_prefix_env_name + str(i) + token = os.getenv(token_name) + gh = Github(token) + rest, _ = gh.rate_limiting + tokens[token] = rest + + return max(tokens.items(), key=lambda x: x[1])[0] diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index c254ad74ae4..34052adecdf 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -9,6 +9,7 @@ from s3_helper import S3Helper from pr_info import PRInfo import shutil import sys +from get_robot_token import get_best_robot_token NAME = 'PVS Studio (actions)' LICENCE_NAME = 'Free license: ClickHouse, Yandex' @@ -83,7 +84,7 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - gh = Github(os.getenv("GITHUB_TOKEN")) + gh = Github(get_best_robot_token()) images_path = os.path.join(temp_path, 'changed_images.json') docker_image = 'clickhouse/pvs-test' diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 95e827671ca..e6bc7259330 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -6,6 +6,7 @@ from pr_info import PRInfo import sys import logging from github import Github +from get_robot_token import get_best_robot_token NAME = 'Run Check (actions)' @@ -113,7 +114,7 @@ if __name__ == "__main__": pr_info = PRInfo(event, need_orgs=True) can_run, description = should_run_checks_for_pr(pr_info) - gh = Github(os.getenv("GITHUB_TOKEN")) + gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 71978379099..0b1d673e628 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,6 +10,7 @@ from s3_helper import S3Helper import time import json from pr_info import PRInfo +from get_robot_token import get_best_robot_token NAME = "Style Check (actions)" @@ -108,7 +109,7 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - gh = Github(os.getenv("GITHUB_TOKEN")) + gh = Github(get_best_robot_token()) images_path = os.path.join(temp_path, 'changed_images.json') docker_image = 'clickhouse/style-test' From 0dcb36df0155182727726bf326eb08510752cc15 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 13:57:41 +0300 Subject: [PATCH 873/950] Remove statuses from actions --- .github/workflows/main.yml | 26 -------------------------- tests/ci/docker_images_check.py | 10 ---------- tests/ci/pvs_check.py | 10 ---------- tests/ci/run_check.py | 7 ------- tests/ci/style_check.py | 11 ----------- 5 files changed, 64 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 60be4368df7..46a66ce98ce 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -17,11 +17,6 @@ jobs: uses: actions/checkout@v2 - name: Labels check run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py - env: - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DockerHubPush: needs: CheckLabels runs-on: [self-hosted] @@ -33,10 +28,6 @@ jobs: env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} - name: Upload images files to artifacts uses: actions/upload-artifact@v2 @@ -58,21 +49,4 @@ jobs: env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py - FinishCheck: - needs: [StyleCheck, DockerHubPush, CheckLabels] - runs-on: [self-hosted] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Finish label - run: cd $GITHUB_WORKSPACE/tests/ci && python3 finish_check.py - env: - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index d1954d70e71..0ddca0718e7 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,7 +8,6 @@ import os from pr_info import PRInfo from github import Github import shutil -from get_robot_token import get_best_robot_token NAME = "Push to Dockerhub (actions)" @@ -168,11 +167,6 @@ def upload_results(s3_client, pr_number, commit_sha, test_results): logging.info("Search result in url %s", url) return url -def get_commit(gh, commit_sha): - repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) - commit = repo.get_commit(commit_sha) - return commit - if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) @@ -223,10 +217,6 @@ if __name__ == "__main__": url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) - gh = Github(get_best_robot_token()) - commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description=description, state=status, target_url=url) - with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: json.dump(result_images, images_file) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 34052adecdf..94e046c0a68 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -9,7 +9,6 @@ from s3_helper import S3Helper from pr_info import PRInfo import shutil import sys -from get_robot_token import get_best_robot_token NAME = 'PVS Studio (actions)' LICENCE_NAME = 'Free license: ClickHouse, Yandex' @@ -40,11 +39,6 @@ def _process_txt_report(path): errors.append(':'.join(line.split('\t')[0:2])) return warnings, errors -def get_commit(gh, commit_sha): - repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) - commit = repo.get_commit(commit_sha) - return commit - def upload_results(s3_client, pr_number, commit_sha, test_results, additional_files): s3_path_prefix = str(pr_number) + "/" + commit_sha + "/" + NAME.lower().replace(' ', '_') additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) @@ -84,8 +78,6 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - gh = Github(get_best_robot_token()) - images_path = os.path.join(temp_path, 'changed_images.json') docker_image = 'clickhouse/pvs-test' if os.path.exists(images_path): @@ -138,8 +130,6 @@ if __name__ == "__main__": report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs) print("::notice ::Report url: {}".format(report_url)) - commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description=description, state=status, target_url=report_url) except Exception as ex: print("Got an exception", ex) sys.exit(1) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index e6bc7259330..3371e5a5720 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -5,8 +5,6 @@ import requests from pr_info import PRInfo import sys import logging -from github import Github -from get_robot_token import get_best_robot_token NAME = 'Run Check (actions)' @@ -114,13 +112,8 @@ if __name__ == "__main__": pr_info = PRInfo(event, need_orgs=True) can_run, description = should_run_checks_for_pr(pr_info) - gh = Github(get_best_robot_token()) - commit = get_commit(gh, pr_info.sha) - url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: print("::notice ::Cannot run") - commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) else: print("::notice ::Can run") - commit.create_status(context=NAME, description=description, state="pending", target_url=url) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 0b1d673e628..c7c25d2a95b 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,7 +10,6 @@ from s3_helper import S3Helper import time import json from pr_info import PRInfo -from get_robot_token import get_best_robot_token NAME = "Style Check (actions)" @@ -79,12 +78,6 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi logging.info("Search result in url %s", url) return url - -def get_commit(gh, commit_sha): - repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) - commit = repo.get_commit(commit_sha) - return commit - def update_check_with_curl(check_id): cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " "--header 'authorization: Bearer {}' " @@ -109,8 +102,6 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - gh = Github(get_best_robot_token()) - images_path = os.path.join(temp_path, 'changed_images.json') docker_image = 'clickhouse/style-test' if os.path.exists(images_path): @@ -141,5 +132,3 @@ if __name__ == "__main__": state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) print("::notice ::Report url: {}".format(report_url)) - commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description=description, state=state, target_url=report_url) From 0d6712532ad00274bbcfa9b20d5832c0123db88c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 14:25:14 +0300 Subject: [PATCH 874/950] Revert "Remove statuses from actions" This reverts commit 0dcb36df0155182727726bf326eb08510752cc15. --- .github/workflows/main.yml | 26 ++++++++++++++++++++++++++ tests/ci/docker_images_check.py | 10 ++++++++++ tests/ci/pvs_check.py | 10 ++++++++++ tests/ci/run_check.py | 7 +++++++ tests/ci/style_check.py | 11 +++++++++++ 5 files changed, 64 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 46a66ce98ce..60be4368df7 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -17,6 +17,11 @@ jobs: uses: actions/checkout@v2 - name: Labels check run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py + env: + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DockerHubPush: needs: CheckLabels runs-on: [self-hosted] @@ -28,6 +33,10 @@ jobs: env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} - name: Upload images files to artifacts uses: actions/upload-artifact@v2 @@ -49,4 +58,21 @@ jobs: env: YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py + FinishCheck: + needs: [StyleCheck, DockerHubPush, CheckLabels] + runs-on: [self-hosted] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: cd $GITHUB_WORKSPACE/tests/ci && python3 finish_check.py + env: + ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} + ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} + ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} + ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0ddca0718e7..d1954d70e71 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,6 +8,7 @@ import os from pr_info import PRInfo from github import Github import shutil +from get_robot_token import get_best_robot_token NAME = "Push to Dockerhub (actions)" @@ -167,6 +168,11 @@ def upload_results(s3_client, pr_number, commit_sha, test_results): logging.info("Search result in url %s", url) return url +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) @@ -217,6 +223,10 @@ if __name__ == "__main__": url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results) + gh = Github(get_best_robot_token()) + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=status, target_url=url) + with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: json.dump(result_images, images_file) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 94e046c0a68..34052adecdf 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -9,6 +9,7 @@ from s3_helper import S3Helper from pr_info import PRInfo import shutil import sys +from get_robot_token import get_best_robot_token NAME = 'PVS Studio (actions)' LICENCE_NAME = 'Free license: ClickHouse, Yandex' @@ -39,6 +40,11 @@ def _process_txt_report(path): errors.append(':'.join(line.split('\t')[0:2])) return warnings, errors +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + def upload_results(s3_client, pr_number, commit_sha, test_results, additional_files): s3_path_prefix = str(pr_number) + "/" + commit_sha + "/" + NAME.lower().replace(' ', '_') additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) @@ -78,6 +84,8 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") + gh = Github(get_best_robot_token()) + images_path = os.path.join(temp_path, 'changed_images.json') docker_image = 'clickhouse/pvs-test' if os.path.exists(images_path): @@ -130,6 +138,8 @@ if __name__ == "__main__": report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs) print("::notice ::Report url: {}".format(report_url)) + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=status, target_url=report_url) except Exception as ex: print("Got an exception", ex) sys.exit(1) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 3371e5a5720..e6bc7259330 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -5,6 +5,8 @@ import requests from pr_info import PRInfo import sys import logging +from github import Github +from get_robot_token import get_best_robot_token NAME = 'Run Check (actions)' @@ -112,8 +114,13 @@ if __name__ == "__main__": pr_info = PRInfo(event, need_orgs=True) can_run, description = should_run_checks_for_pr(pr_info) + gh = Github(get_best_robot_token()) + commit = get_commit(gh, pr_info.sha) + url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: print("::notice ::Cannot run") + commit.create_status(context=NAME, description=description, state="failure", target_url=url) sys.exit(1) else: print("::notice ::Can run") + commit.create_status(context=NAME, description=description, state="pending", target_url=url) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index c7c25d2a95b..0b1d673e628 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -10,6 +10,7 @@ from s3_helper import S3Helper import time import json from pr_info import PRInfo +from get_robot_token import get_best_robot_token NAME = "Style Check (actions)" @@ -78,6 +79,12 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi logging.info("Search result in url %s", url) return url + +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + def update_check_with_curl(check_id): cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " "--header 'authorization: Bearer {}' " @@ -102,6 +109,8 @@ if __name__ == "__main__": aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") + gh = Github(get_best_robot_token()) + images_path = os.path.join(temp_path, 'changed_images.json') docker_image = 'clickhouse/style-test' if os.path.exists(images_path): @@ -132,3 +141,5 @@ if __name__ == "__main__": state, description, test_results, additional_files = process_result(temp_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files) print("::notice ::Report url: {}".format(report_url)) + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description=description, state=state, target_url=report_url) From f607a5d0ab12d6aa8d38c602493996392b906187 Mon Sep 17 00:00:00 2001 From: vesslanjin Date: Wed, 20 Oct 2021 07:40:06 -0400 Subject: [PATCH 875/950] remove branch in ColumnDecimal, same as pull req #29881 Co-authored-by: Zhu Jasper jasper.zhu@intel.com Signed-off-by: vesslanjin --- src/Columns/ColumnDecimal.cpp | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index d8c5ced4b6b..dc236fafbd9 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -237,6 +237,26 @@ ColumnPtr ColumnDecimal::filter(const IColumn::Filter & filt, ssize_t result_ const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); +#ifdef __SSE2__ + static constexpr size_t SIMD_BYTES = 16; + const __m128i zero16 = _mm_setzero_si128(); + const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_sse) + { + UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); + mask = ~mask; + while (mask) + { + size_t index = __builtin_ctz(mask); + res_data.push_back(*(data_pos + index)); + mask = mask & (mask - 1); + } + filt_pos += SIMD_BYTES; + data_pos += SIMD_BYTES; + } +#endif + while (filt_pos < filt_end) { if (*filt_pos) From d120d3720af4d59013d696536a0aa6f950ae6394 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 14:48:27 +0300 Subject: [PATCH 876/950] Trying aws secrets --- .github/workflows/main.yml | 25 ------------------------- tests/ci/docker_images_check.py | 9 +++------ tests/ci/get_robot_token.py | 14 ++++++++++---- tests/ci/pvs_check.py | 8 +------- tests/ci/s3_helper.py | 7 ++++--- tests/ci/style_check.py | 8 +------- 6 files changed, 19 insertions(+), 52 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 60be4368df7..7f20206a7b3 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -17,11 +17,6 @@ jobs: uses: actions/checkout@v2 - name: Labels check run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py - env: - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} DockerHubPush: needs: CheckLabels runs-on: [self-hosted] @@ -30,14 +25,6 @@ jobs: uses: actions/checkout@v2 - name: Images check run: cd $GITHUB_WORKSPACE/tests/ci && python3 docker_images_check.py - env: - YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} - YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} - DOCKER_ROBOT_PASSWORD: ${{ secrets.DOCKER_ROBOT_PASSWORD }} - name: Upload images files to artifacts uses: actions/upload-artifact@v2 with: @@ -55,13 +42,6 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Style Check - env: - YANDEX_S3_ACCESS_KEY_ID: ${{ secrets.YANDEX_S3_ACCESS_KEY_ID }} - YANDEX_S3_ACCESS_SECRET_KEY: ${{ secrets.YANDEX_S3_ACCESS_SECRET_KEY }} - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels] @@ -71,8 +51,3 @@ jobs: uses: actions/checkout@v2 - name: Finish label run: cd $GITHUB_WORKSPACE/tests/ci && python3 finish_check.py - env: - ROBOT_TOKEN_0: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN }} - ROBOT_TOKEN_1: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_1 }} - ROBOT_TOKEN_2: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_2 }} - ROBOT_TOKEN_3: ${{ secrets.ROBOT_CLICKHOUSE_PERSONAL_ACCESS_TOKEN_3 }} diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index d1954d70e71..9bd3f431429 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -8,7 +8,7 @@ import os from pr_info import PRInfo from github import Github import shutil -from get_robot_token import get_best_robot_token +from get_robot_token import get_best_robot_token, get_parameter_from_ssm NAME = "Push to Dockerhub (actions)" @@ -177,7 +177,7 @@ if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') - dockerhub_password = os.getenv('DOCKER_ROBOT_PASSWORD') + dockerhub_password = get_parameter_from_ssm('dockerhub_robot_password') if os.path.exists(temp_path): shutil.rmtree(temp_path) @@ -213,10 +213,7 @@ if __name__ == "__main__": if len(description) >= 140: description = description[:136] + "..." - aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") - aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - - s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + s3_helper = S3Helper('https://s3.amazonaws.com') s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') status, test_results = process_test_results(s3_helper, images_processing_result, s3_path_prefix) diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py index 75b688e5b44..73fdcd670fd 100644 --- a/tests/ci/get_robot_token.py +++ b/tests/ci/get_robot_token.py @@ -1,12 +1,18 @@ #!/usr/bin/env python3 -import os +import boto3 from github import Github -def get_best_robot_token(token_prefix_env_name="ROBOT_TOKEN_", total_tokens=4): +def get_parameter_from_ssm(name, decrypt=True, client=None): + if not client: + client = boto3.client('ssm') + return client.get_parameter(Name=name, WithDecryption=decrypt)['Parameter']['Value'] + +def get_best_robot_token(token_prefix_env_name="github_robot_token_", total_tokens=4): + client = boto3.client('ssm') tokens = {} - for i in range(total_tokens): + for i in range(1, total_tokens + 1): token_name = token_prefix_env_name + str(i) - token = os.getenv(token_name) + token = get_parameter_from_ssm(token_name, True, client) gh = Github(token) rest, _ = gh.rate_limiting tokens[token] = rest diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 34052adecdf..f8b1b58f307 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -81,9 +81,6 @@ if __name__ == "__main__": # this check modify repository so copy it to the temp directory logging.info("Repo copy path %s", repo_path) - aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") - aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - gh = Github(get_best_robot_token()) images_path = os.path.join(temp_path, 'changed_images.json') @@ -98,10 +95,7 @@ if __name__ == "__main__": logging.info("Got docker image %s", docker_image) - if not aws_secret_key_id or not aws_secret_key: - logging.info("No secrets, will not upload anything to S3") - - s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + s3_helper = S3Helper('https://s3.amazonaws.com') licence_key = os.getenv('PVS_STUDIO_KEY') cmd = f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENCE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index b9ae0de6e02..4054f650223 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -6,6 +6,7 @@ import boto3 from botocore.exceptions import ClientError, BotoCoreError from multiprocessing.dummy import Pool from compress_files import compress_file_fast +from get_robot_token import get_parameter_from_ssm def _md5(fname): hash_md5 = hashlib.md5() @@ -27,8 +28,8 @@ def _flatten_list(lst): class S3Helper(object): - def __init__(self, host, aws_access_key_id, aws_secret_access_key): - self.session = boto3.session.Session(aws_access_key_id=aws_access_key_id, aws_secret_access_key=aws_secret_access_key) + def __init__(self, host): + self.session = boto3.session.Session() self.client = self.session.client('s3', endpoint_url=host) def _upload_file_to_s3(self, bucket_name, file_path, s3_path): @@ -55,7 +56,7 @@ class S3Helper(object): self.client.upload_file(file_path, bucket_name, s3_path, ExtraArgs=metadata) logging.info("Upload {} to {}. Meta: {}".format(file_path, s3_path, metadata)) - return "https://storage.yandexcloud.net/{bucket}/{path}".format(bucket=bucket_name, path=s3_path) + return "https://s3.amazonaws.com/{bucket}/{path}".format(bucket=bucket_name, path=s3_path) def upload_test_report_to_s3(self, file_path, s3_path): return self._upload_file_to_s3('clickhouse-test-reports', file_path, s3_path) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 0b1d673e628..4a8cde70bc2 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -106,9 +106,6 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) - aws_secret_key_id = os.getenv("YANDEX_S3_ACCESS_KEY_ID", "") - aws_secret_key = os.getenv("YANDEX_S3_ACCESS_SECRET_KEY", "") - gh = Github(get_best_robot_token()) images_path = os.path.join(temp_path, 'changed_images.json') @@ -132,10 +129,7 @@ if __name__ == "__main__": else: raise Exception(f"Cannot pull dockerhub for image {docker_image}") - if not aws_secret_key_id or not aws_secret_key: - logging.info("No secrets, will not upload anything to S3") - - s3_helper = S3Helper('https://storage.yandexcloud.net', aws_access_key_id=aws_secret_key_id, aws_secret_access_key=aws_secret_key) + s3_helper = S3Helper('https://s3.amazonaws.com') subprocess.check_output(f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) From 6ea04b2ea66c50f815d102345dc08afc7d56ca85 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 14:52:03 +0300 Subject: [PATCH 877/950] Fix region --- tests/ci/get_robot_token.py | 4 ++-- tests/ci/s3_helper.py | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py index 73fdcd670fd..db37ee311c5 100644 --- a/tests/ci/get_robot_token.py +++ b/tests/ci/get_robot_token.py @@ -4,11 +4,11 @@ from github import Github def get_parameter_from_ssm(name, decrypt=True, client=None): if not client: - client = boto3.client('ssm') + client = boto3.client('ssm', region_name='us-east-1') return client.get_parameter(Name=name, WithDecryption=decrypt)['Parameter']['Value'] def get_best_robot_token(token_prefix_env_name="github_robot_token_", total_tokens=4): - client = boto3.client('ssm') + client = boto3.client('ssm', region_name='us-east-1') tokens = {} for i in range(1, total_tokens + 1): token_name = token_prefix_env_name + str(i) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 4054f650223..3c930f26634 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -29,7 +29,7 @@ def _flatten_list(lst): class S3Helper(object): def __init__(self, host): - self.session = boto3.session.Session() + self.session = boto3.session.Session(region_name='us-east-1') self.client = self.session.client('s3', endpoint_url=host) def _upload_file_to_s3(self, bucket_name, file_path, s3_path): From ac358d08a6477715448a6b0b2834e19b3430d613 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 15:33:20 +0300 Subject: [PATCH 878/950] Fix style check --- src/Common/examples/executable_udf.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/examples/executable_udf.cpp b/src/Common/examples/executable_udf.cpp index 78a248fcddf..8d2d9f7314e 100644 --- a/src/Common/examples/executable_udf.cpp +++ b/src/Common/examples/executable_udf.cpp @@ -24,11 +24,13 @@ int main(int argc, char **argv) size_t rows = 0; char dummy; - while (!read_buffer.eof()) { + while (!read_buffer.eof()) + { readIntText(rows, read_buffer); readChar(dummy, read_buffer); - for (size_t i = 0; i < rows; ++i) { + for (size_t i = 0; i < rows; ++i) + { readString(buffer, read_buffer); readChar(dummy, read_buffer); From 54d37204265a523a6aa26fbfd6506186d45333da Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 16:08:13 +0300 Subject: [PATCH 879/950] ASTDropFunctionQuery formatting fix --- src/Parsers/ASTDropFunctionQuery.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTDropFunctionQuery.cpp b/src/Parsers/ASTDropFunctionQuery.cpp index 0a46940e73d..47665aa52f9 100644 --- a/src/Parsers/ASTDropFunctionQuery.cpp +++ b/src/Parsers/ASTDropFunctionQuery.cpp @@ -12,10 +12,13 @@ ASTPtr ASTDropFunctionQuery::clone() const void ASTDropFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState &, IAST::FormatStateStacked) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP FUNCTION " << (settings.hilite ? hilite_none : ""); - settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "DROP FUNCTION "; + if (if_exists) - settings.ostr << "IF EXISTS"; + settings.ostr << "IF EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : ""); } } From 240895fba765180d3e6021b9db9ba8590a580c98 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 16 Oct 2021 23:07:55 +0800 Subject: [PATCH 880/950] Fix projections with JOIN alias columns --- src/Interpreters/ExpressionAnalyzer.cpp | 3 ++- .../0_stateless/01710_projection_with_joins.reference | 0 tests/queries/0_stateless/01710_projection_with_joins.sql | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01710_projection_with_joins.reference create mode 100644 tests/queries/0_stateless/01710_projection_with_joins.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index a1f7a3c71e5..3cb3c1b47ab 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -890,9 +890,10 @@ static std::unique_ptr buildJoinedPlan( * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. * - this function shows the expression JOIN _data1. + * - JOIN tables will need aliases to correctly resolve USING clause. */ auto interpreter = interpretSubquery( - join_element.table_expression, context, original_right_columns, query_options.copy().setWithAllColumns()); + join_element.table_expression, context, original_right_columns, query_options.copy().setWithAllColumns().ignoreAlias(false)); auto joined_plan = std::make_unique(); interpreter->buildQueryPlan(*joined_plan); { diff --git a/tests/queries/0_stateless/01710_projection_with_joins.reference b/tests/queries/0_stateless/01710_projection_with_joins.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_with_joins.sql b/tests/queries/0_stateless/01710_projection_with_joins.sql new file mode 100644 index 00000000000..97dc396f362 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_with_joins.sql @@ -0,0 +1,6 @@ +drop table if exists t; + +create table t (s UInt16, l UInt16, projection p (select s, l order by l)) engine MergeTree order by s; + +set allow_experimental_projection_optimization=1; +select s from t join (select toUInt16(1) as s) x using (s); From 2f615e9176d8bb75da7a9c69586e9f5e4d81c46e Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 19 Oct 2021 22:09:43 +0800 Subject: [PATCH 881/950] Use original_query for projection analysis --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/SelectQueryInfo.h | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 35c8c32c65b..59d0b88bae7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -272,6 +272,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; + query_info.original_query = query_ptr->clone(); initSettings(); const Settings & settings = context->getSettingsRef(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee55a6826c3..6503e6dca12 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4531,7 +4531,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( if (!settings.allow_experimental_projection_optimization || query_info.ignore_projections || query_info.is_projection_query) return false; - const auto & query_ptr = query_info.query; + const auto & query_ptr = query_info.original_query; if (auto * select = query_ptr->as(); select) { diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index b08818a2baa..fe7b22d331b 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -133,6 +133,7 @@ struct SelectQueryInfo { ASTPtr query; ASTPtr view_query; /// Optimized VIEW query + ASTPtr original_query; /// Unmodified query for projection analysis /// Cluster for the query. ClusterPtr cluster; From 367e58357ad6c7e286860640ec02161f3e338801 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 20 Oct 2021 13:43:39 +0000 Subject: [PATCH 882/950] Better test --- .../0_stateless/01236_graphite_mt.reference | 344 ++++++++++++++++++ .../queries/0_stateless/01236_graphite_mt.sql | 4 + 2 files changed, 348 insertions(+) diff --git a/tests/queries/0_stateless/01236_graphite_mt.reference b/tests/queries/0_stateless/01236_graphite_mt.reference index a30d2495265..0f2e8e81377 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.reference +++ b/tests/queries/0_stateless/01236_graphite_mt.reference @@ -342,3 +342,347 @@ 2 sum_2 98950 1 940 2 sum_2 108950 1 1040 2 sum_2 70170 1 1140 +1 max_1 9 1 0 +1 max_1 19 1 10 +1 max_1 29 1 20 +1 max_1 39 1 30 +1 max_1 49 1 40 +1 max_1 59 1 50 +1 max_1 69 1 60 +1 max_1 79 1 70 +1 max_1 89 1 80 +1 max_1 99 1 90 +1 max_1 109 1 100 +1 max_1 119 1 110 +1 max_1 129 1 120 +1 max_1 139 1 130 +1 max_1 149 1 140 +1 max_1 159 1 150 +1 max_1 169 1 160 +1 max_1 179 1 170 +1 max_1 189 1 180 +1 max_1 199 1 190 +1 max_1 209 1 200 +1 max_1 219 1 210 +1 max_1 229 1 220 +1 max_1 239 1 230 +1 max_1 249 1 240 +1 max_1 259 1 250 +1 max_1 269 1 260 +1 max_1 279 1 270 +1 max_1 289 1 280 +1 max_1 299 1 290 +1 max_1 39 1 0 +1 max_1 139 1 40 +1 max_1 239 1 140 +1 max_1 339 1 240 +1 max_1 439 1 340 +1 max_1 539 1 440 +1 max_1 639 1 540 +1 max_1 739 1 640 +1 max_1 839 1 740 +1 max_1 939 1 840 +1 max_1 1039 1 940 +1 max_1 1139 1 1040 +1 max_1 1199 1 1140 +1 max_2 9 1 0 +1 max_2 19 1 10 +1 max_2 29 1 20 +1 max_2 39 1 30 +1 max_2 49 1 40 +1 max_2 59 1 50 +1 max_2 69 1 60 +1 max_2 79 1 70 +1 max_2 89 1 80 +1 max_2 99 1 90 +1 max_2 109 1 100 +1 max_2 119 1 110 +1 max_2 129 1 120 +1 max_2 139 1 130 +1 max_2 149 1 140 +1 max_2 159 1 150 +1 max_2 169 1 160 +1 max_2 179 1 170 +1 max_2 189 1 180 +1 max_2 199 1 190 +1 max_2 209 1 200 +1 max_2 219 1 210 +1 max_2 229 1 220 +1 max_2 239 1 230 +1 max_2 249 1 240 +1 max_2 259 1 250 +1 max_2 269 1 260 +1 max_2 279 1 270 +1 max_2 289 1 280 +1 max_2 299 1 290 +1 max_2 39 1 0 +1 max_2 139 1 40 +1 max_2 239 1 140 +1 max_2 339 1 240 +1 max_2 439 1 340 +1 max_2 539 1 440 +1 max_2 639 1 540 +1 max_2 739 1 640 +1 max_2 839 1 740 +1 max_2 939 1 840 +1 max_2 1039 1 940 +1 max_2 1139 1 1040 +1 max_2 1199 1 1140 +1 sum_1 45 1 0 +1 sum_1 145 1 10 +1 sum_1 245 1 20 +1 sum_1 345 1 30 +1 sum_1 445 1 40 +1 sum_1 545 1 50 +1 sum_1 645 1 60 +1 sum_1 745 1 70 +1 sum_1 845 1 80 +1 sum_1 945 1 90 +1 sum_1 1045 1 100 +1 sum_1 1145 1 110 +1 sum_1 1245 1 120 +1 sum_1 1345 1 130 +1 sum_1 1445 1 140 +1 sum_1 1545 1 150 +1 sum_1 1645 1 160 +1 sum_1 1745 1 170 +1 sum_1 1845 1 180 +1 sum_1 1945 1 190 +1 sum_1 2045 1 200 +1 sum_1 2145 1 210 +1 sum_1 2245 1 220 +1 sum_1 2345 1 230 +1 sum_1 2445 1 240 +1 sum_1 2545 1 250 +1 sum_1 2645 1 260 +1 sum_1 2745 1 270 +1 sum_1 2845 1 280 +1 sum_1 2945 1 290 +1 sum_1 780 1 0 +1 sum_1 8950 1 40 +1 sum_1 18950 1 140 +1 sum_1 28950 1 240 +1 sum_1 38950 1 340 +1 sum_1 48950 1 440 +1 sum_1 58950 1 540 +1 sum_1 68950 1 640 +1 sum_1 78950 1 740 +1 sum_1 88950 1 840 +1 sum_1 98950 1 940 +1 sum_1 108950 1 1040 +1 sum_1 70170 1 1140 +1 sum_2 45 1 0 +1 sum_2 145 1 10 +1 sum_2 245 1 20 +1 sum_2 345 1 30 +1 sum_2 445 1 40 +1 sum_2 545 1 50 +1 sum_2 645 1 60 +1 sum_2 745 1 70 +1 sum_2 845 1 80 +1 sum_2 945 1 90 +1 sum_2 1045 1 100 +1 sum_2 1145 1 110 +1 sum_2 1245 1 120 +1 sum_2 1345 1 130 +1 sum_2 1445 1 140 +1 sum_2 1545 1 150 +1 sum_2 1645 1 160 +1 sum_2 1745 1 170 +1 sum_2 1845 1 180 +1 sum_2 1945 1 190 +1 sum_2 2045 1 200 +1 sum_2 2145 1 210 +1 sum_2 2245 1 220 +1 sum_2 2345 1 230 +1 sum_2 2445 1 240 +1 sum_2 2545 1 250 +1 sum_2 2645 1 260 +1 sum_2 2745 1 270 +1 sum_2 2845 1 280 +1 sum_2 2945 1 290 +1 sum_2 780 1 0 +1 sum_2 8950 1 40 +1 sum_2 18950 1 140 +1 sum_2 28950 1 240 +1 sum_2 38950 1 340 +1 sum_2 48950 1 440 +1 sum_2 58950 1 540 +1 sum_2 68950 1 640 +1 sum_2 78950 1 740 +1 sum_2 88950 1 840 +1 sum_2 98950 1 940 +1 sum_2 108950 1 1040 +1 sum_2 70170 1 1140 +2 max_1 9 1 0 +2 max_1 19 1 10 +2 max_1 29 1 20 +2 max_1 39 1 30 +2 max_1 49 1 40 +2 max_1 59 1 50 +2 max_1 69 1 60 +2 max_1 79 1 70 +2 max_1 89 1 80 +2 max_1 99 1 90 +2 max_1 109 1 100 +2 max_1 119 1 110 +2 max_1 129 1 120 +2 max_1 139 1 130 +2 max_1 149 1 140 +2 max_1 159 1 150 +2 max_1 169 1 160 +2 max_1 179 1 170 +2 max_1 189 1 180 +2 max_1 199 1 190 +2 max_1 209 1 200 +2 max_1 219 1 210 +2 max_1 229 1 220 +2 max_1 239 1 230 +2 max_1 249 1 240 +2 max_1 259 1 250 +2 max_1 269 1 260 +2 max_1 279 1 270 +2 max_1 289 1 280 +2 max_1 299 1 290 +2 max_1 39 1 0 +2 max_1 139 1 40 +2 max_1 239 1 140 +2 max_1 339 1 240 +2 max_1 439 1 340 +2 max_1 539 1 440 +2 max_1 639 1 540 +2 max_1 739 1 640 +2 max_1 839 1 740 +2 max_1 939 1 840 +2 max_1 1039 1 940 +2 max_1 1139 1 1040 +2 max_1 1199 1 1140 +2 max_2 9 1 0 +2 max_2 19 1 10 +2 max_2 29 1 20 +2 max_2 39 1 30 +2 max_2 49 1 40 +2 max_2 59 1 50 +2 max_2 69 1 60 +2 max_2 79 1 70 +2 max_2 89 1 80 +2 max_2 99 1 90 +2 max_2 109 1 100 +2 max_2 119 1 110 +2 max_2 129 1 120 +2 max_2 139 1 130 +2 max_2 149 1 140 +2 max_2 159 1 150 +2 max_2 169 1 160 +2 max_2 179 1 170 +2 max_2 189 1 180 +2 max_2 199 1 190 +2 max_2 209 1 200 +2 max_2 219 1 210 +2 max_2 229 1 220 +2 max_2 239 1 230 +2 max_2 249 1 240 +2 max_2 259 1 250 +2 max_2 269 1 260 +2 max_2 279 1 270 +2 max_2 289 1 280 +2 max_2 299 1 290 +2 max_2 39 1 0 +2 max_2 139 1 40 +2 max_2 239 1 140 +2 max_2 339 1 240 +2 max_2 439 1 340 +2 max_2 539 1 440 +2 max_2 639 1 540 +2 max_2 739 1 640 +2 max_2 839 1 740 +2 max_2 939 1 840 +2 max_2 1039 1 940 +2 max_2 1139 1 1040 +2 max_2 1199 1 1140 +2 sum_1 45 1 0 +2 sum_1 145 1 10 +2 sum_1 245 1 20 +2 sum_1 345 1 30 +2 sum_1 445 1 40 +2 sum_1 545 1 50 +2 sum_1 645 1 60 +2 sum_1 745 1 70 +2 sum_1 845 1 80 +2 sum_1 945 1 90 +2 sum_1 1045 1 100 +2 sum_1 1145 1 110 +2 sum_1 1245 1 120 +2 sum_1 1345 1 130 +2 sum_1 1445 1 140 +2 sum_1 1545 1 150 +2 sum_1 1645 1 160 +2 sum_1 1745 1 170 +2 sum_1 1845 1 180 +2 sum_1 1945 1 190 +2 sum_1 2045 1 200 +2 sum_1 2145 1 210 +2 sum_1 2245 1 220 +2 sum_1 2345 1 230 +2 sum_1 2445 1 240 +2 sum_1 2545 1 250 +2 sum_1 2645 1 260 +2 sum_1 2745 1 270 +2 sum_1 2845 1 280 +2 sum_1 2945 1 290 +2 sum_1 780 1 0 +2 sum_1 8950 1 40 +2 sum_1 18950 1 140 +2 sum_1 28950 1 240 +2 sum_1 38950 1 340 +2 sum_1 48950 1 440 +2 sum_1 58950 1 540 +2 sum_1 68950 1 640 +2 sum_1 78950 1 740 +2 sum_1 88950 1 840 +2 sum_1 98950 1 940 +2 sum_1 108950 1 1040 +2 sum_1 70170 1 1140 +2 sum_2 45 1 0 +2 sum_2 145 1 10 +2 sum_2 245 1 20 +2 sum_2 345 1 30 +2 sum_2 445 1 40 +2 sum_2 545 1 50 +2 sum_2 645 1 60 +2 sum_2 745 1 70 +2 sum_2 845 1 80 +2 sum_2 945 1 90 +2 sum_2 1045 1 100 +2 sum_2 1145 1 110 +2 sum_2 1245 1 120 +2 sum_2 1345 1 130 +2 sum_2 1445 1 140 +2 sum_2 1545 1 150 +2 sum_2 1645 1 160 +2 sum_2 1745 1 170 +2 sum_2 1845 1 180 +2 sum_2 1945 1 190 +2 sum_2 2045 1 200 +2 sum_2 2145 1 210 +2 sum_2 2245 1 220 +2 sum_2 2345 1 230 +2 sum_2 2445 1 240 +2 sum_2 2545 1 250 +2 sum_2 2645 1 260 +2 sum_2 2745 1 270 +2 sum_2 2845 1 280 +2 sum_2 2945 1 290 +2 sum_2 780 1 0 +2 sum_2 8950 1 40 +2 sum_2 18950 1 140 +2 sum_2 28950 1 240 +2 sum_2 38950 1 340 +2 sum_2 48950 1 440 +2 sum_2 58950 1 540 +2 sum_2 68950 1 640 +2 sum_2 78950 1 740 +2 sum_2 88950 1 840 +2 sum_2 98950 1 940 +2 sum_2 108950 1 1040 +2 sum_2 70170 1 1140 diff --git a/tests/queries/0_stateless/01236_graphite_mt.sql b/tests/queries/0_stateless/01236_graphite_mt.sql index 1d531f88ecb..0ec905fa0a8 100644 --- a/tests/queries/0_stateless/01236_graphite_mt.sql +++ b/tests/queries/0_stateless/01236_graphite_mt.sql @@ -34,4 +34,8 @@ WITH dates AS select key, Path, Value, Version, col from test_graphite final order by key, Path, Time desc; +optimize table test_graphite final; + +select key, Path, Value, Version, col from test_graphite order by key, Path, Time desc; + drop table test_graphite; From f47aec6751c830f9d3f8a099d813e9225646a143 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 17:52:28 +0300 Subject: [PATCH 883/950] Fixed tests --- tests/queries/0_stateless/01856_create_function.sql | 1 - .../02098_sql_user_defined_functions_aliases.reference | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01856_create_function.sql b/tests/queries/0_stateless/01856_create_function.sql index 10f87f4a3df..cdc4baad1af 100644 --- a/tests/queries/0_stateless/01856_create_function.sql +++ b/tests/queries/0_stateless/01856_create_function.sql @@ -4,7 +4,6 @@ CREATE FUNCTION 01856_test_function_0 AS (a, b, c) -> a * b * c; SELECT 01856_test_function_0(2, 3, 4); SELECT isConstant(01856_test_function_0(1, 2, 3)); DROP FUNCTION 01856_test_function_0; -CREATE FUNCTION 01856_test_function_1 AS (a, b) -> a || b || c; --{serverError 47} CREATE FUNCTION 01856_test_function_1 AS (a, b) -> 01856_test_function_1(a, b) + 01856_test_function_1(a, b); --{serverError 611} CREATE FUNCTION cast AS a -> a + 1; --{serverError 609} CREATE FUNCTION sum AS (a, b) -> a + b; --{serverError 609} diff --git a/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference index 8ab2f6d0ac6..45a4fb75db8 100644 --- a/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference +++ b/tests/queries/0_stateless/02098_sql_user_defined_functions_aliases.reference @@ -1 +1 @@ -8 4 +8 From ff48017f4a35ee222f34d88f25df78eedf30f322 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 20 Oct 2021 16:04:02 +0300 Subject: [PATCH 884/950] SQLUserDefinedFunctions support CREATE OR REPLACE, CREATE IF NOT EXISTS --- .../InterpreterCreateFunctionQuery.cpp | 20 +++++++++++++++---- src/Interpreters/InterpreterFactory.cpp | 2 +- .../UserDefinedSQLFunctionFactory.cpp | 16 ++++++++++----- .../UserDefinedSQLFunctionFactory.h | 12 ++++++++++- .../UserDefinedSQLObjectsLoader.cpp | 10 +++++----- .../UserDefinedSQLObjectsLoader.h | 2 +- src/Parsers/ASTCreateFunctionQuery.cpp | 13 +++++++++++- src/Parsers/ASTCreateFunctionQuery.h | 3 +++ src/Parsers/ParserCreateFunctionQuery.cpp | 17 +++++++++++++++- ...ined_functions_create_or_replace.reference | 4 ++++ ...er_defined_functions_create_or_replace.sql | 13 ++++++++++++ ...d_functions_create_if_not_exists.reference | 1 + ...defined_functions_create_if_not_exists.sql | 8 ++++++++ 13 files changed, 102 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.reference create mode 100644 tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.sql create mode 100644 tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.reference create mode 100644 tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.sql diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index ccb5f4040ec..39fec4a941c 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -31,20 +31,32 @@ BlockIO InterpreterCreateFunctionQuery::execute() if (!create_function_query) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected CREATE FUNCTION query"); + auto & user_defined_function_factory = UserDefinedSQLFunctionFactory::instance(); + auto & function_name = create_function_query->function_name; + + bool if_not_exists = create_function_query->if_not_exists; + bool replace = create_function_query->or_replace; + + create_function_query->if_not_exists = false; + create_function_query->or_replace = false; + + if (if_not_exists && user_defined_function_factory.tryGet(function_name) != nullptr) + return {}; + validateFunction(create_function_query->function_core, function_name); - UserDefinedSQLFunctionFactory::instance().registerFunction(function_name, query_ptr); + user_defined_function_factory.registerFunction(function_name, query_ptr, replace); - if (!persist_function) + if (persist_function) { try { - UserDefinedSQLObjectsLoader::instance().storeObject(current_context, UserDefinedSQLObjectType::Function, function_name, *query_ptr); + UserDefinedSQLObjectsLoader::instance().storeObject(current_context, UserDefinedSQLObjectType::Function, function_name, *query_ptr, replace); } catch (Exception & exception) { - UserDefinedSQLFunctionFactory::instance().unregisterFunction(function_name); + user_defined_function_factory.unregisterFunction(function_name); exception.addMessage(fmt::format("while storing user defined function {} on disk", backQuote(function_name))); throw; } diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index 54307ae848b..fcf5f19aef6 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -278,7 +278,7 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, ContextMut } else if (query->as()) { - return std::make_unique(query, context, false /*is_internal*/); + return std::make_unique(query, context, true /*persist_function*/); } else if (query->as()) { diff --git a/src/Interpreters/UserDefinedSQLFunctionFactory.cpp b/src/Interpreters/UserDefinedSQLFunctionFactory.cpp index 1d2a80305c6..f036741ca21 100644 --- a/src/Interpreters/UserDefinedSQLFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedSQLFunctionFactory.cpp @@ -19,7 +19,7 @@ UserDefinedSQLFunctionFactory & UserDefinedSQLFunctionFactory::instance() return result; } -void UserDefinedSQLFunctionFactory::registerFunction(const String & function_name, ASTPtr create_function_query) +void UserDefinedSQLFunctionFactory::registerFunction(const String & function_name, ASTPtr create_function_query, bool replace) { if (FunctionFactory::instance().hasNameOrAlias(function_name)) throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The function '{}' already exists", function_name); @@ -29,11 +29,17 @@ void UserDefinedSQLFunctionFactory::registerFunction(const String & function_nam std::lock_guard lock(mutex); - auto [_, inserted] = function_name_to_create_query.emplace(function_name, std::move(create_function_query)); + auto [it, inserted] = function_name_to_create_query.emplace(function_name, create_function_query); + if (!inserted) - throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, - "The function name '{}' is not unique", - function_name); + { + if (replace) + it->second = std::move(create_function_query); + else + throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, + "The function name '{}' is not unique", + function_name); + } } void UserDefinedSQLFunctionFactory::unregisterFunction(const String & function_name) diff --git a/src/Interpreters/UserDefinedSQLFunctionFactory.h b/src/Interpreters/UserDefinedSQLFunctionFactory.h index 6838c2f9892..6487b951705 100644 --- a/src/Interpreters/UserDefinedSQLFunctionFactory.h +++ b/src/Interpreters/UserDefinedSQLFunctionFactory.h @@ -10,21 +10,31 @@ namespace DB { +/// Factory for SQLUserDefinedFunctions class UserDefinedSQLFunctionFactory : public IHints<1, UserDefinedSQLFunctionFactory> { public: static UserDefinedSQLFunctionFactory & instance(); - void registerFunction(const String & function_name, ASTPtr create_function_query); + /** Register function for function_name in factory for specified create_function_query. + * If replace = true and function with function_name already exists replace it with create_function_query. + * Otherwise throws exception. + */ + void registerFunction(const String & function_name, ASTPtr create_function_query, bool replace); + /// Unregister function for function_name void unregisterFunction(const String & function_name); + /// Get function create query for function_name. If no function registered with function_name throws exception. ASTPtr get(const String & function_name) const; + /// Get function create query for function_name. If no function registered with function_name return nullptr. ASTPtr tryGet(const String & function_name) const; + /// Check if function with function_name registered. bool has(const String & function_name) const; + /// Get all user defined functions registered names. std::vector getAllRegisteredNames() const override; private: diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp index e4eb97f3002..a71f1f0799c 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.cpp +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.cpp @@ -69,7 +69,7 @@ void UserDefinedSQLObjectsLoader::loadUserDefinedObject(ContextPtr context, User 0, context->getSettingsRef().max_parser_depth); - InterpreterCreateFunctionQuery interpreter(ast, context, true /*is internal*/); + InterpreterCreateFunctionQuery interpreter(ast, context, false /*persist_function*/); interpreter.execute(); } } @@ -111,7 +111,7 @@ void UserDefinedSQLObjectsLoader::loadObjects(ContextPtr context) } } -void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast) +void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast, bool replace) { if (unlikely(!enable_persistence)) return; @@ -127,7 +127,7 @@ void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQL } } - if (std::filesystem::exists(file_path)) + if (!replace && std::filesystem::exists(file_path)) throw Exception(ErrorCodes::OBJECT_ALREADY_STORED_ON_DISK, "User defined object {} already stored on disk", backQuote(file_path)); LOG_DEBUG(log, "Storing object {} to file {}", backQuote(object_name), file_path); @@ -135,9 +135,9 @@ void UserDefinedSQLObjectsLoader::storeObject(ContextPtr context, UserDefinedSQL WriteBufferFromOwnString create_statement_buf; formatAST(ast, create_statement_buf, false); writeChar('\n', create_statement_buf); - String create_statement = create_statement_buf.str(); - WriteBufferFromFile out(file_path, create_statement.size(), O_WRONLY | O_CREAT | O_EXCL); + + WriteBufferFromFile out(file_path, create_statement.size()); writeString(create_statement, out); out.next(); if (context->getSettingsRef().fsync_metadata) diff --git a/src/Interpreters/UserDefinedSQLObjectsLoader.h b/src/Interpreters/UserDefinedSQLObjectsLoader.h index 17493933f21..2e747f67a8d 100644 --- a/src/Interpreters/UserDefinedSQLObjectsLoader.h +++ b/src/Interpreters/UserDefinedSQLObjectsLoader.h @@ -21,7 +21,7 @@ public: UserDefinedSQLObjectsLoader(); void loadObjects(ContextPtr context); - void storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast); + void storeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name, const IAST & ast, bool replace); void removeObject(ContextPtr context, UserDefinedSQLObjectType object_type, const String & object_name); /// For ClickHouse local if path is not set we can disable loader. diff --git a/src/Parsers/ASTCreateFunctionQuery.cpp b/src/Parsers/ASTCreateFunctionQuery.cpp index 0b3991ddc44..4e1e7de660d 100644 --- a/src/Parsers/ASTCreateFunctionQuery.cpp +++ b/src/Parsers/ASTCreateFunctionQuery.cpp @@ -12,7 +12,18 @@ ASTPtr ASTCreateFunctionQuery::clone() const void ASTCreateFunctionQuery::formatImpl(const IAST::FormatSettings & settings, IAST::FormatState & state, IAST::FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE FUNCTION " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "CREATE "; + + if (or_replace) + settings.ostr << "OR REPLACE "; + + settings.ostr << "FUNCTION "; + + if (if_not_exists) + settings.ostr << "IF NOT EXISTS "; + + settings.ostr << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_identifier : "") << backQuoteIfNeed(function_name) << (settings.hilite ? hilite_none : ""); settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : ""); function_core->formatImpl(settings, state, frame); diff --git a/src/Parsers/ASTCreateFunctionQuery.h b/src/Parsers/ASTCreateFunctionQuery.h index 3adddad8fbd..a58fe64c435 100644 --- a/src/Parsers/ASTCreateFunctionQuery.h +++ b/src/Parsers/ASTCreateFunctionQuery.h @@ -12,6 +12,9 @@ public: String function_name; ASTPtr function_core; + bool or_replace = false; + bool if_not_exists = false; + String getID(char) const override { return "CreateFunctionQuery"; } ASTPtr clone() const override; diff --git a/src/Parsers/ParserCreateFunctionQuery.cpp b/src/Parsers/ParserCreateFunctionQuery.cpp index fbfd02415e7..5d84b6bc2dc 100644 --- a/src/Parsers/ParserCreateFunctionQuery.cpp +++ b/src/Parsers/ParserCreateFunctionQuery.cpp @@ -1,10 +1,12 @@ +#include + #include #include #include #include #include #include -#include + namespace DB { @@ -13,6 +15,8 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp { ParserKeyword s_create("CREATE"); ParserKeyword s_function("FUNCTION"); + ParserKeyword s_or_replace("OR REPLACE"); + ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserIdentifier function_name_p; ParserKeyword s_as("AS"); ParserLambdaExpression lambda_p; @@ -20,12 +24,21 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp ASTPtr function_name; ASTPtr function_core; + bool or_replace = false; + bool if_not_exists = false; + if (!s_create.ignore(pos, expected)) return false; + if (s_or_replace.ignore(pos, expected)) + or_replace = true; + if (!s_function.ignore(pos, expected)) return false; + if (!or_replace && s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + if (!function_name_p.parse(pos, function_name, expected)) return false; @@ -40,6 +53,8 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp create_function_query->function_name = function_name->as().name(); create_function_query->function_core = function_core; + create_function_query->or_replace = or_replace; + create_function_query->if_not_exists = if_not_exists; return true; } diff --git a/tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.reference b/tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.reference new file mode 100644 index 00000000000..437cc81afba --- /dev/null +++ b/tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.reference @@ -0,0 +1,4 @@ +CREATE FUNCTION `02101_test_function` AS x -> (x + 1) +2 +CREATE FUNCTION `02101_test_function` AS x -> (x + 2) +3 diff --git a/tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.sql b/tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.sql new file mode 100644 index 00000000000..7b0ad311bd4 --- /dev/null +++ b/tests/queries/0_stateless/02101_sql_user_defined_functions_create_or_replace.sql @@ -0,0 +1,13 @@ +-- Tags: no-parallel + +CREATE OR REPLACE FUNCTION 02101_test_function AS x -> x + 1; + +SELECT create_query FROM system.functions WHERE name = '02101_test_function'; +SELECT 02101_test_function(1); + +CREATE OR REPLACE FUNCTION 02101_test_function AS x -> x + 2; + +SELECT create_query FROM system.functions WHERE name = '02101_test_function'; +SELECT 02101_test_function(1); + +DROP FUNCTION 02101_test_function; diff --git a/tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.reference b/tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.sql b/tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.sql new file mode 100644 index 00000000000..092fa660cb0 --- /dev/null +++ b/tests/queries/0_stateless/02102_sql_user_defined_functions_create_if_not_exists.sql @@ -0,0 +1,8 @@ +-- Tags: no-parallel + +CREATE FUNCTION IF NOT EXISTS 02102_test_function AS x -> x + 1; +SELECT 02102_test_function(1); + +CREATE FUNCTION 02102_test_function AS x -> x + 1; --{serverError 609} +CREATE FUNCTION IF NOT EXISTS 02102_test_function AS x -> x + 1; +DROP FUNCTION 02102_test_function; From 7cb6cddf6dfe6c76a8baa397ef587aeb3edee852 Mon Sep 17 00:00:00 2001 From: gyuton Date: Wed, 20 Oct 2021 19:52:18 +0300 Subject: [PATCH 885/950] Initial draft --- .../functions/splitting-merging-functions.md | 37 ++++++++++++++++++ .../functions/splitting-merging-functions.md | 38 +++++++++++++++++++ 2 files changed, 75 insertions(+) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 718d5a977b9..bc06193612a 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -270,3 +270,40 @@ Result: │ [['abc','123'],['8','"hkl"']] │ └───────────────────────────────────────────────────────────────────────┘ ``` + +## ngrams {#ngrams} + +Splits the ASCII string into n-grams of `ngramsize` symbols. + +**Syntax** + +``` sql +ngrams(string, ngramsize) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). +- `ngramsize` — The size of an n-gram. [UInt](../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- Array with n-grams. + +Type: [Array](../../sql-reference/data-types/array.md)([FixedString](../../sql-reference/data-types/fixedstring.md)). + +**Example** + +Query: + +``` sql +SELECT ngrams('ClickHouse', 3); +``` + +Result: + +``` text +┌─ngrams('ClickHouse', 3)───────────────────────────┐ +│ ['Cli','lic','ick','ckH','kHo','Hou','ous','use'] │ +└───────────────────────────────────────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index efe74dba043..9b9aab4c437 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -232,3 +232,41 @@ SELECT alphaTokens('abca1abc'); │ ['abca','abc'] │ └─────────────────────────┘ ``` + +## ngrams {#ngrams} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов. + +**Синтаксис** + +``` sql +ngrams(string, ngramsize) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). +- `ngramsize` — размер n-грамм. [UInt](../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Массив с n-граммами. + +Тип: [Array](../../sql-reference/data-types/array.md)([FixedString](../../sql-reference/data-types/fixedstring.md)). + +**Пример** + +Запрос: + +``` sql +SELECT ngrams('ClickHouse', 3); +``` + +Результат: + +``` text +┌─ngrams('ClickHouse', 3)───────────────────────────┐ +│ ['Cli','lic','ick','ckH','kHo','Hou','ous','use'] │ +└───────────────────────────────────────────────────┘ +``` + From 220791b21c31cfa73a8e8d656c44080d38d80033 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 17:15:07 +0000 Subject: [PATCH 886/950] Fix --- src/Interpreters/Context.cpp | 7 +++++-- src/Interpreters/Context.h | 2 -- src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 -- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d01057d5259..03216b717d3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -267,6 +267,9 @@ struct ContextSharedPart bool shutdown_called = false; + /// Has background executors for MergeTree tables been initialized? + bool is_background_executors_initialized = false; + Stopwatch uptime_watch; Context::ApplicationType application_type = Context::ApplicationType::SERVER; @@ -2974,7 +2977,7 @@ void Context::setAsynchronousInsertQueue(const std::shared_ptris_background_executors_initialized) return; const size_t max_merges_and_mutations = getSettingsRef().background_pool_size * getSettingsRef().background_merges_mutations_concurrency_ratio; @@ -3024,7 +3027,7 @@ void Context::initializeBackgroundExecutorsIfNeeded() LOG_INFO(shared->log, "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}", getSettingsRef().background_common_pool_size, getSettingsRef().background_common_pool_size); - is_background_executors_initialized = true; + shared->is_background_executors_initialized = true; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 85b4fbf23d5..8518275e529 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -293,8 +293,6 @@ private: /// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL). bool is_internal_query = false; - /// Has background executors for MergeTree tables been initialized? - bool is_background_executors_initialized = false; public: diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 218d990e0ce..982acfe62a4 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -301,8 +301,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) name_part = name_part.substr(strlen("Replicated")); - args.getContext()->getGlobalContext()->initializeBackgroundExecutorsIfNeeded(); - MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; From b0752b43ac9529b4f744af72e32806c4240824b4 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 20 Oct 2021 20:20:14 +0300 Subject: [PATCH 887/950] Add the getOSKernelVersion function MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал английскую версию функции getOSKernelVersion. --- .../functions/other-functions.md | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 6864ba7705b..a7f81ab8325 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2427,3 +2427,39 @@ Type: [UInt32](../../sql-reference/data-types/int-uint.md). **See Also** - [shardNum()](#shard-num) function example also contains `shardCount()` function call. + +## getOSKernelVersion {#getoskernelversion} + +Returns string with the current OS kernel version. + +**Syntax** + +``` sql +getOSKernelVersion() +``` + +**Arguments** + +- None. + +**Returned value** + +- The current OS kernel version. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT getOSKernelVersion(); +``` + +Result: + +``` text +┌─getOSKernelVersion()────┐ +│ Linux 4.15.0-55-generic │ +└─────────────────────────┘ +``` From 360ec76c29a6bbfc550ed3286661becc5d40642b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jo=C3=A3o=20Figueiredo?= Date: Wed, 20 Oct 2021 22:35:17 +0200 Subject: [PATCH 888/950] Grammar suggestions to distributed.md * fixed some typos. * improved wording of some statements. --- docs/en/engines/table-engines/special/distributed.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 6593a5dc17f..708dab6fb7d 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -5,7 +5,7 @@ toc_title: Distributed # Distributed Table Engine {#distributed} -Tables with Distributed engine do not store any data by their own, but allow distributed query processing on multiple servers. +Tables with Distributed engine do not store any data of their own, but allow distributed query processing on multiple servers. Reading is automatically parallelized. During a read, the table indexes on remote servers are used, if there are any. The Distributed engine accepts parameters: @@ -167,20 +167,20 @@ If this parameter is set to `true`, the write operation selects the first health If it is set to `false` (the default), data is written to all replicas. In essence, this means that the Distributed table replicates data itself. This is worse than using replicated tables, because the consistency of replicas is not checked, and over time they will contain slightly different data. -To select the shard that a row of data is sent to, the sharding expression is analyzed, and its remainder is taken from dividing it by the total weight of the shards. The row is sent to the shard that corresponds to the half-interval of the remainders from `prev_weight` to `prev_weights + weight`, where `prev_weights` is the total weight of the shards with the smallest number, and `weight` is the weight of this shard. For example, if there are two shards, and the first has a weight of 9 while the second has a weight of 10, the row will be sent to the first shard for the remainders from the range \[0, 9), and to the second for the remainders from the range \[9, 19). +To select the shard that a row of data is sent to, the sharding expression is analyzed, and its remainder is taken from dividing it by the total weight of the shards. The row is sent to the shard that corresponds to the half-interval of the remainders from `prev_weights` to `prev_weights + weight`, where `prev_weights` is the total weight of the shards with the smallest number, and `weight` is the weight of this shard. For example, if there are two shards, and the first has a weight of 9 while the second has a weight of 10, the row will be sent to the first shard for the remainders from the range \[0, 9), and to the second for the remainders from the range \[9, 19). The sharding expression can be any expression from constants and table columns that returns an integer. For example, you can use the expression `rand()` for random distribution of data, or `UserID` for distribution by the remainder from dividing the user’s ID (then the data of a single user will reside on a single shard, which simplifies running IN and JOIN by users). If one of the columns is not distributed evenly enough, you can wrap it in a hash function: intHash64(UserID). -A simple reminder from the division is a limited solution for sharding and isn’t always appropriate. It works for medium and large volumes of data (dozens of servers), but not for very large volumes of data (hundreds of servers or more). In the latter case, use the sharding scheme required by the subject area, rather than using entries in Distributed tables. +A simple remainder from the division is a limited solution for sharding and isn’t always appropriate. It works for medium and large volumes of data (dozens of servers), but not for very large volumes of data (hundreds of servers or more). In the latter case, use the sharding scheme required by the subject area, rather than using entries in Distributed tables. -SELECT queries are sent to all the shards and work regardless of how data is distributed across the shards (they can be distributed completely randomly). When you add a new shard, you do not have to transfer the old data to it. You can write new data with a heavier weight – the data will be distributed slightly unevenly, but queries will work correctly and efficiently. +SELECT queries are sent to all the shards and work regardless of how data is distributed across the shards (they can be distributed completely randomly). When you add a new shard, you do not have to transfer old data into it. Instead, you can write new data to it by using a heavier weight – the data will be distributed slightly unevenly, but queries will work correctly and efficiently. You should be concerned about the sharding scheme in the following cases: - Queries are used that require joining data (IN or JOIN) by a specific key. If data is sharded by this key, you can use local IN or JOIN instead of GLOBAL IN or GLOBAL JOIN, which is much more efficient. - A large number of servers is used (hundreds or more) with a large number of small queries (queries of individual clients - websites, advertisers, or partners). In order for the small queries to not affect the entire cluster, it makes sense to locate data for a single client on a single shard. Alternatively, as we’ve done in Yandex.Metrica, you can set up bi-level sharding: divide the entire cluster into “layers”, where a layer may consist of multiple shards. Data for a single client is located on a single layer, but shards can be added to a layer as necessary, and data is randomly distributed within them. Distributed tables are created for each layer, and a single shared distributed table is created for global queries. -Data is written asynchronously. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The period for sending data is managed by the [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) and [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting. +Data is written asynchronously. When inserted in the table, the data block is just written to the local file system. The data is sent to the remote servers in the background as soon as possible. The periodicity for sending data is managed by the [distributed_directory_monitor_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_sleep_time_ms) and [distributed_directory_monitor_max_sleep_time_ms](../../../operations/settings/settings.md#distributed_directory_monitor_max_sleep_time_ms) settings. The `Distributed` engine sends each file with inserted data separately, but you can enable batch sending of files with the [distributed_directory_monitor_batch_inserts](../../../operations/settings/settings.md#distributed_directory_monitor_batch_inserts) setting. This setting improves cluster performance by better utilizing local server and network resources. You should check whether data is sent successfully by checking the list of files (data waiting to be sent) in the table directory: `/var/lib/clickhouse/data/database/table/`. The number of threads performing background tasks can be set by [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting. If the server ceased to exist or had a rough restart (for example, after a device failure) after an INSERT to a Distributed table, the inserted data might be lost. If a damaged data part is detected in the table directory, it is transferred to the `broken` subdirectory and no longer used. From c96b884951a32c4d1637ea9053be88c2cbd08846 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 23:40:29 +0300 Subject: [PATCH 889/950] Fix --- src/Functions/FunctionSQLJSON.h | 41 +++++----- .../0_stateless/01889_sql_json_functions.sql | 74 +++++++++---------- 2 files changed, 56 insertions(+), 59 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index d59ead2ebf5..4c27f090fb2 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -50,36 +50,33 @@ public: throw Exception{"JSONPath functions require at least 2 arguments", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION}; } - const auto & first_column = arguments[0]; + const auto & json_column = arguments[0]; - /// Check 1 argument: must be of type String (JSONPath) - if (!isString(first_column.type)) + if (!isString(json_column.type)) { throw Exception( - "JSONPath functions require 1 argument to be JSONPath of type string, illegal type: " + first_column.type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - /// Check 1 argument: must be const (JSONPath) - if (!isColumnConst(*first_column.column)) - { - throw Exception("1 argument (JSONPath) must be const", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - const auto & second_column = arguments[1]; - - /// Check 2 argument: must be of type String (JSON) - if (!isString(second_column.type)) - { - throw Exception( - "JSONPath functions require 2 argument to be JSON of string, illegal type: " + second_column.type->getName(), + "JSONPath functions require first argument to be JSON of string, illegal type: " + json_column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - const ColumnPtr & arg_jsonpath = first_column.column; + const auto & json_path_column = arguments[1]; + + if (!isString(json_path_column.type)) + { + throw Exception( + "JSONPath functions require second argument to be JSONPath of type string, illegal type: " + json_path_column.type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + if (!isColumnConst(*json_path_column.column)) + { + throw Exception("Second argument (JSONPath) must be constant string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + const ColumnPtr & arg_jsonpath = json_path_column.column; const auto * arg_jsonpath_const = typeid_cast(arg_jsonpath.get()); const auto * arg_jsonpath_string = typeid_cast(arg_jsonpath_const->getDataColumnPtr().get()); - const ColumnPtr & arg_json = second_column.column; + const ColumnPtr & arg_json = json_column.column; const auto * col_json_const = typeid_cast(arg_json.get()); const auto * col_json_string = typeid_cast(col_json_const ? col_json_const->getDataColumnPtr().get() : arg_json.get()); @@ -152,7 +149,7 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 0c4f3aff9a3..087f029e635 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -1,46 +1,46 @@ -- Tags: no-fasttest SELECT '--JSON_VALUE--'; -SELECT JSON_VALUE('$', '{"hello":1}'); -- root is a complex object => default value (empty string) -SELECT JSON_VALUE('$.hello', '{"hello":1}'); -SELECT JSON_VALUE('$.hello', '{"hello":1.2}'); -SELECT JSON_VALUE('$.hello', '{"hello":true}'); -SELECT JSON_VALUE('$.hello', '{"hello":"world"}'); -SELECT JSON_VALUE('$.hello', '{"hello":null}'); -SELECT JSON_VALUE('$.hello', '{"hello":["world","world2"]}'); -SELECT JSON_VALUE('$.hello', '{"hello":{"world":"!"}}'); -SELECT JSON_VALUE('$.hello', '{hello:world}'); -- invalid json => default value (empty string) -SELECT JSON_VALUE('$.hello', ''); +SELECT JSON_VALUE('{"hello":1}', '$'); -- root is a complex object => default value (empty string) +SELECT JSON_VALUE('{"hello":1}', '$.hello'); +SELECT JSON_VALUE('{"hello":1.2}', '$.hello'); +SELECT JSON_VALUE('{"hello":true}', '$.hello'); +SELECT JSON_VALUE('{"hello":"world"}', '$.hello'); +SELECT JSON_VALUE('{"hello":null}', '$.hello'); +SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); +SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); +SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) +SELECT JSON_VALUE('', '$.hello'); SELECT '--JSON_QUERY--'; -SELECT JSON_QUERY('$', '{"hello":1}'); -SELECT JSON_QUERY('$.hello', '{"hello":1}'); -SELECT JSON_QUERY('$.hello', '{"hello":1.2}'); -SELECT JSON_QUERY('$.hello', '{"hello":true}'); -SELECT JSON_QUERY('$.hello', '{"hello":"world"}'); -SELECT JSON_QUERY('$.hello', '{"hello":null}'); -SELECT JSON_QUERY('$.hello', '{"hello":["world","world2"]}'); -SELECT JSON_QUERY('$.hello', '{"hello":{"world":"!"}}'); -SELECT JSON_QUERY('$.hello', '{hello:{"world":"!"}}}'); -- invalid json => default value (empty string) -SELECT JSON_QUERY('$.hello', ''); -SELECT JSON_QUERY('$.array[*][0 to 2, 4]', '{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}'); +SELECT JSON_QUERY('{"hello":1}', '$'); +SELECT JSON_QUERY('{"hello":1}', '$.hello'); +SELECT JSON_QUERY('{"hello":1.2}', '$.hello'); +SELECT JSON_QUERY('{"hello":true}', '$.hello'); +SELECT JSON_QUERY('{"hello":"world"}', '$.hello'); +SELECT JSON_QUERY('{"hello":null}', '$.hello'); +SELECT JSON_QUERY('{"hello":["world","world2"]}', '$.hello'); +SELECT JSON_QUERY('{"hello":{"world":"!"}}', '$.hello'); +SELECT JSON_QUERY( '{hello:{"world":"!"}}}', '$.hello'); -- invalid json => default value (empty string) +SELECT JSON_QUERY('', '$.hello'); +SELECT JSON_QUERY('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); SELECT '--JSON_EXISTS--'; -SELECT JSON_EXISTS('$', '{"hello":1}'); -SELECT JSON_EXISTS('$', ''); -SELECT JSON_EXISTS('$', '{}'); -SELECT JSON_EXISTS('$.hello', '{"hello":1}'); -SELECT JSON_EXISTS('$.world', '{"hello":1,"world":2}'); -SELECT JSON_EXISTS('$.world', '{"hello":{"world":1}}'); -SELECT JSON_EXISTS('$.hello.world', '{"hello":{"world":1}}'); -SELECT JSON_EXISTS('$.hello', '{hello:world}'); -- invalid json => default value (zero integer) -SELECT JSON_EXISTS('$.hello', ''); -SELECT JSON_EXISTS('$.hello[*]', '{"hello":["world"]}'); -SELECT JSON_EXISTS('$.hello[0]', '{"hello":["world"]}'); -SELECT JSON_EXISTS('$.hello[1]', '{"hello":["world"]}'); -SELECT JSON_EXISTS('$.a[*].b', '{"a":[{"b":1},{"c":2}]}'); -SELECT JSON_EXISTS('$.a[*].f', '{"a":[{"b":1},{"c":2}]}'); -SELECT JSON_EXISTS('$.a[*][0].h', '{"a":[[{"b":1}, {"g":1}],[{"h":1},{"y":1}]]}'); +SELECT JSON_EXISTS('{"hello":1}', '$'); +SELECT JSON_EXISTS('', '$'); +SELECT JSON_EXISTS('{}', '$'); +SELECT JSON_EXISTS('{"hello":1}', '$.hello'); +SELECT JSON_EXISTS('{"hello":1,"world":2}', '$.world'); +SELECT JSON_EXISTS('{"hello":{"world":1}}', '$.world'); +SELECT JSON_EXISTS('{"hello":{"world":1}}', '$.hello.world'); +SELECT JSON_EXISTS('{hello:world}', '$.hello'); -- invalid json => default value (zero integer) +SELECT JSON_EXISTS('', '$.hello'); +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[*]'); +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[0]'); +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[1]'); +SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].b'); +SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].f'); +SELECT JSON_EXISTS('{"a":[[{"b":1}, {"g":1}],[{"h":1},{"y":1}]]}', '$.a[*][0].h'); SELECT '--MANY ROWS--'; DROP TABLE IF EXISTS 01889_sql_json; @@ -48,5 +48,5 @@ CREATE TABLE 01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY INSERT INTO 01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}'); INSERT INTO 01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}'); INSERT INTO 01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}'); -SELECT id, JSON_QUERY('$.friends[0 to 2]', json) FROM 01889_sql_json ORDER BY id; +SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id; DROP TABLE 01889_sql_json; From 422b0ba54705bdfb9908066a281de0e959950fba Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Wed, 20 Oct 2021 16:12:22 -0600 Subject: [PATCH 890/950] Update release date and add training link --- website/blog/en/2021/clickhouse-v21.10-released.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.10-released.md b/website/blog/en/2021/clickhouse-v21.10-released.md index b49bb284f90..ed0aab88017 100644 --- a/website/blog/en/2021/clickhouse-v21.10-released.md +++ b/website/blog/en/2021/clickhouse-v21.10-released.md @@ -15,6 +15,8 @@ Let's highlight some of these new exciting new capabilities in 21.10: * Instead of logging every query (which can be a lot of logs!), you can now log a random sample of your queries. The number of queries logged is determined by defining a specified probability between 0.0 (no queries logged) and 1.0 (all queries logged) using the new `log_queries_probability` setting. * Positional arguments are now available in your GROUP BY, ORDER BY and LIMIT BY clauses. For example, `SELECT foo, bar, baz FROM my_table ORDER BY 2,3` orders the results by whatever the bar and baz columns (no need to specify column names twice!) +We're also thrilled to announce some new free training available to you in our Learn ClickHouse portal: [https://clickhouse.com/learn/lessons/whatsnew-clickhouse-21.10/](https://clickhouse.com/learn/lessons/whatsnew-clickhouse-21.10/) + We're always listening for new ideas, and we're happy to welcome new contributors to the ClickHouse project. Whether for submitting code or improving our documentation and examples, please get involved by sending us a pull request or submitting an issue. Our beginner developers contribution guide will help you get started: [https://clickhouse.com/docs/en/development/developer-instruction/](https://clickhouse.com/docs/en/development/developer-instruction/) @@ -22,6 +24,6 @@ We're always listening for new ideas, and we're happy to welcome new contributor Release 21.10 -Release Date: 2021-10-21 +Release Date: 2021-10-17 Release Notes: [21.10](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) From cbf397fa4e218d826f23e2480f7112879507450a Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Wed, 20 Oct 2021 17:17:39 -0600 Subject: [PATCH 891/950] Fix horizontal scroll bar --- website/css/main.css | 2 +- website/src/scss/components/_page.scss | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/website/css/main.css b/website/css/main.css index 229b74cb3f7..1d0b6f5c116 100644 --- a/website/css/main.css +++ b/website/css/main.css @@ -1 +1 @@ -@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:flex;justify-content:center}.btns .btn+.btn{margin-left:24px}.btns .btn-lg+.btn-lg{margin-left:40px}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(10%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(60%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3,.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s}}@media screen and (min-width:980px){.case-study-card .col-lg-3{transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3,.case-study-card.is-open .col-lg-auto{opacity:1;transform:none}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{transition-delay:.4s}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page{overflow:hidden;width:100vw}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;overflow:hidden;position:relative;width:100%}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{color:#495057;list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file +@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:flex;justify-content:center}.btns .btn+.btn{margin-left:24px}.btns .btn-lg+.btn-lg{margin-left:40px}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(10%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(60%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3,.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s}}@media screen and (min-width:980px){.case-study-card .col-lg-3{transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3,.case-study-card.is-open .col-lg-auto{opacity:1;transform:none}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{transition-delay:.4s}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{color:#495057;list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file diff --git a/website/src/scss/components/_page.scss b/website/src/scss/components/_page.scss index 926f9656b73..a411ff7a15c 100644 --- a/website/src/scss/components/_page.scss +++ b/website/src/scss/components/_page.scss @@ -1,4 +1,4 @@ .page { overflow: hidden; - width: 100vw; + width: 100%; } From a223526f451d2bdb32041a137ca9f5bbf31dcc28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 23:29:35 +0000 Subject: [PATCH 892/950] Fix --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 5ac4180ec27..2adb359eae5 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -216,7 +216,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " "attnotnull AS not_null, attndims AS dims " "FROM pg_attribute " - "WHERE attrelid = {}::regclass " + "WHERE attrelid = (SELECT oid FROM pg_class WHERE relname = {})" "AND NOT attisdropped AND attnum > 0", quoteString(postgres_table_name)); table.columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, false); From e679e952c89e1af257a2fecd0a1e960ef1b67401 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 21 Oct 2021 10:16:11 +0800 Subject: [PATCH 893/950] Let's check without projection just in case --- tests/queries/0_stateless/01710_projection_with_joins.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01710_projection_with_joins.sql b/tests/queries/0_stateless/01710_projection_with_joins.sql index 97dc396f362..fcd1c586fa3 100644 --- a/tests/queries/0_stateless/01710_projection_with_joins.sql +++ b/tests/queries/0_stateless/01710_projection_with_joins.sql @@ -2,5 +2,7 @@ drop table if exists t; create table t (s UInt16, l UInt16, projection p (select s, l order by l)) engine MergeTree order by s; -set allow_experimental_projection_optimization=1; -select s from t join (select toUInt16(1) as s) x using (s); +select s from t join (select toUInt16(1) as s) x using (s) settings allow_experimental_projection_optimization = 1; +select s from t join (select toUInt16(1) as s) x using (s) settings allow_experimental_projection_optimization = 0; + +drop table t; From 650a79a90771d64b6e70ac8708316923a5fa60aa Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Oct 2021 10:08:08 +0300 Subject: [PATCH 894/950] SQLUserDefinedFunctions composition fix --- src/Interpreters/UserDefinedSQLFunctionVisitor.cpp | 5 ++++- src/Parsers/ASTFunction.cpp | 8 ++++++-- ..._sql_user_defined_functions_composition.reference | 2 ++ .../02103_sql_user_defined_functions_composition.sql | 12 ++++++++++++ 4 files changed, 24 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference create mode 100644 tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql diff --git a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp index 8df6932b153..d9ac53097ab 100644 --- a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data &) +void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data & data) { auto * function = ast->as(); if (!function) @@ -27,7 +27,10 @@ void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data &) auto result = tryToReplaceFunction(*function); if (result) + { ast = result; + visit(ast, data); + } } bool UserDefinedSQLFunctionMatcher::needChildVisit(const ASTPtr &, const ASTPtr &) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 3c78c4060d6..093baca375c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -440,6 +440,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (!written && 0 == strcmp(name.c_str(), "lambda")) { + /// Special case: zero elements tuple in lhs of lambda is printed as (). /// Special case: one-element tuple in lhs of lambda is printed as its element. if (frame.need_parens) @@ -449,9 +450,12 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (first_arg_func && first_arg_func->name == "tuple" && first_arg_func->arguments - && first_arg_func->arguments->children.size() == 1) + && (first_arg_func->arguments->children.size() == 1 || first_arg_func->arguments->children.size() == 0)) { - first_arg_func->arguments->children[0]->formatImpl(settings, state, nested_need_parens); + if (first_arg_func->arguments->children.size() == 1) + first_arg_func->arguments->children[0]->formatImpl(settings, state, nested_need_parens); + else + settings.ostr << "()"; } else arguments->children[0]->formatImpl(settings, state, nested_need_parens); diff --git a/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference new file mode 100644 index 00000000000..51993f072d5 --- /dev/null +++ b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference @@ -0,0 +1,2 @@ +2 +2 diff --git a/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql new file mode 100644 index 00000000000..3d34413b9d3 --- /dev/null +++ b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql @@ -0,0 +1,12 @@ +-- Tags: no-parallel + +CREATE FUNCTION 02103_test_function AS x -> x + 1; +CREATE FUNCTION 02103_test_function_with_nested_function_empty_args AS () -> 02103_test_function(1); +CREATE FUNCTION 02103_test_function_with_nested_function_arg AS (x) -> 02103_test_function(x); + +SELECT 02103_test_function_with_nested_function_empty_args(); +SELECT 02103_test_function_with_nested_function_arg(1); + +DROP FUNCTION 02103_test_function_with_nested_function_empty_args; +DROP FUNCTION 02103_test_function_with_nested_function_arg; +DROP FUNCTION 02103_test_function; From ed0bb2cec22d06eb3ff44ddb611038b982396306 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Oct 2021 14:02:51 +0300 Subject: [PATCH 895/950] Fix deadlock on ALTER with scalar subquery to the same table --- src/Interpreters/MutationsInterpreter.cpp | 6 +++--- src/Interpreters/TreeRewriter.cpp | 5 +++-- src/Interpreters/TreeRewriter.h | 3 ++- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index bfa8a06c618..e7b09b6917f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -442,7 +442,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (column.default_desc.kind == ColumnDefaultKind::Materialized) { auto query = column.default_desc.expression->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); for (const String & dependency : syntax_result->requiredSourceColumns()) { if (updated_columns.count(dependency)) @@ -583,7 +583,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS); auto query = (*it).expression_list_ast->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); const auto required_columns = syntax_result->requiredSourceColumns(); for (const auto & column : required_columns) dependencies.emplace(column, ColumnDependency::SKIP_INDEX); @@ -784,7 +784,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot); + auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot, false, true, dry_run); if (context->hasQueryContext()) for (const auto & it : syntax_result->getScalars()) context->getQueryContext()->addScalar(it.first, it.second); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9bcddb6b982..8f7846678ad 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1100,7 +1100,8 @@ TreeRewriterResultPtr TreeRewriter::analyze( ConstStoragePtr storage, const StorageMetadataPtr & metadata_snapshot, bool allow_aggregations, - bool allow_self_aliases) const + bool allow_self_aliases, + bool dry_run) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); @@ -1112,7 +1113,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases); /// Executing scalar subqueries. Column defaults could be a scalar subquery. - executeScalarSubqueries(query, getContext(), 0, result.scalars, false); + executeScalarSubqueries(query, getContext(), 0, result.scalars, dry_run); if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 98201f35216..2430976825f 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -110,7 +110,8 @@ public: ConstStoragePtr storage = {}, const StorageMetadataPtr & metadata_snapshot = {}, bool allow_aggregations = false, - bool allow_self_aliases = true) const; + bool allow_self_aliases = true, + bool dry_run = false) const; /// Analyze and rewrite select query TreeRewriterResultPtr analyzeSelect( From b720b244efc8f2d9c9bebf3af96b0c3b1908bae4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Oct 2021 14:17:53 +0300 Subject: [PATCH 896/950] Add test alter_scalar_circular_deadlock --- ...100_alter_scalar_circular_deadlock.reference | 2 ++ .../02100_alter_scalar_circular_deadlock.sql | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference create mode 100644 tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql new file mode 100644 index 00000000000..5fde10d5d09 --- /dev/null +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS foo; + +CREATE TABLE foo (ts DateTime, x UInt64) +ENGINE = MergeTree PARTITION BY toYYYYMMDD(ts) +ORDER BY (ts); + +INSERT INTO foo (ts, x) SELECT toDateTime('2020-01-01 00:05:00'), number from system.numbers_mt LIMIT 10; + +SET mutations_sync = 1; + +ALTER TABLE foo UPDATE x = 1 WHERE x = (SELECT x from foo WHERE x = 4); + +SELECT sum(x) == 42 FROM foo; + +ALTER TABLE foo UPDATE x = 1 WHERE x IN (SELECT x FROM foo WHERE x != 0); + +SELECT sum(x) == 9 FROM foo; From 2ef25184d0ac658e8e05681317a5d642648a9fe1 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:19:48 +0300 Subject: [PATCH 897/950] Update argmax.md --- docs/ru/sql-reference/aggregate-functions/reference/argmax.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md index 84419523beb..bdf7b1d1df6 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md @@ -6,8 +6,6 @@ toc_priority: 106 Вычисляет значение `arg` при максимальном значении `val`. Если есть несколько разных значений `arg` для максимальных значений `val`, возвращает первое попавшееся из таких значений. -Если функции передан кортеж, то будет выведен кортеж с максимальным значением `val`. Удобно использовать для работы с [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md). - **Синтаксис** ``` sql From a43d293d12320545119570089da3b00585a03123 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:20:25 +0300 Subject: [PATCH 898/950] Update argmax.md --- docs/zh/sql-reference/aggregate-functions/reference/argmax.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md index 0c82cb8a4bb..ed3d2d07a58 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md @@ -6,8 +6,6 @@ toc_priority: 106 计算 `val` 最大值对应的 `arg` 值。 如果 `val` 最大值存在几个不同的 `arg` 值,输出遇到的第一个值。 -这个函数的Tuple版本将返回 `val` 最大值对应的元组。本函数适合和 `SimpleAggregateFunction` 搭配使用。 - **语法** ``` sql From 3afb070b15613f2f99ed25b4e6479f86c194ff9c Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:20:52 +0300 Subject: [PATCH 899/950] Update argmin.md --- docs/ru/sql-reference/aggregate-functions/reference/argmin.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md index 4d2363abe6d..dd923061943 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md @@ -6,8 +6,6 @@ toc_priority: 105 Вычисляет значение `arg` при минимальном значении `val`. Если есть несколько разных значений `arg` для минимальных значений `val`, возвращает первое попавшееся из таких значений. -Если функции передан кортеж, то будет выведен кортеж с минимальным значением `val`. Удобно использовать для работы с [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md). - **Синтаксис** ``` sql From a48da6294971be5f0a6e31aa11a95e13543dd742 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:21:33 +0300 Subject: [PATCH 900/950] Update argmin.md --- docs/zh/sql-reference/aggregate-functions/reference/argmin.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md index 6c2839062eb..8d987300ad4 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md @@ -8,8 +8,6 @@ toc_priority: 105 计算 `val` 最小值对应的 `arg` 值。 如果 `val` 最小值存在几个不同的 `arg` 值,输出遇到的第一个(`arg`)值。 -这个函数的Tuple版本将返回 `val` 最小值对应的tuple。本函数适合和`SimpleAggregateFunction`搭配使用。 - **示例:** 输入表: From 8885748a6c0c033bc946b4f1d0b9e677b0539d0c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Oct 2021 17:25:53 +0300 Subject: [PATCH 901/950] StorageDictionary fix potential configuration race --- src/Storages/StorageDictionary.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 349d894b2f1..51a73eb511e 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -212,11 +212,20 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) auto old_table_id = getStorageID(); IStorage::renameInMemory(new_table_id); - if (configuration) + bool has_configuration = false; { - configuration->setString("dictionary.database", new_table_id.database_name); - configuration->setString("dictionary.name", new_table_id.table_name); + std::lock_guard lock(dictionary_config_mutex); + if (configuration) + { + has_configuration = true; + configuration->setString("dictionary.database", new_table_id.database_name); + configuration->setString("dictionary.name", new_table_id.table_name); + } + } + + if (has_configuration) + { const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName()); From 039af96b61b4293618242a42d473b725a41c0323 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Oct 2021 18:03:16 +0300 Subject: [PATCH 902/950] Set execute_scalar_subqueries in MutationsInterpreter only for MergeTree and prepareInterpreterSelectQuery --- src/Interpreters/MutationsInterpreter.cpp | 9 ++++++--- src/Interpreters/TreeRewriter.cpp | 4 ++-- src/Interpreters/TreeRewriter.h | 2 +- ...100_alter_scalar_circular_deadlock.reference | 2 ++ .../02100_alter_scalar_circular_deadlock.sql | 17 +++++++++++++++++ 5 files changed, 28 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e7b09b6917f..057f7d7caab 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -442,7 +443,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (column.default_desc.kind == ColumnDefaultKind::Materialized) { auto query = column.default_desc.expression->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns); for (const String & dependency : syntax_result->requiredSourceColumns()) { if (updated_columns.count(dependency)) @@ -583,7 +584,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS); auto query = (*it).expression_list_ast->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns); const auto required_columns = syntax_result->requiredSourceColumns(); for (const auto & column : required_columns) dependencies.emplace(column, ColumnDependency::SKIP_INDEX); @@ -784,7 +785,9 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot, false, true, dry_run); + bool execute_scalar_subqueries = !dynamic_cast(storage.get()) || !dry_run; + auto syntax_result = TreeRewriter(context).analyze( + all_asts, all_columns, storage, metadata_snapshot, false, true, execute_scalar_subqueries); if (context->hasQueryContext()) for (const auto & it : syntax_result->getScalars()) context->getQueryContext()->addScalar(it.first, it.second); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8f7846678ad..e0968b7fce4 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1101,7 +1101,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( const StorageMetadataPtr & metadata_snapshot, bool allow_aggregations, bool allow_self_aliases, - bool dry_run) const + bool execute_scalar_subqueries) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); @@ -1113,7 +1113,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases); /// Executing scalar subqueries. Column defaults could be a scalar subquery. - executeScalarSubqueries(query, getContext(), 0, result.scalars, dry_run); + executeScalarSubqueries(query, getContext(), 0, result.scalars, !execute_scalar_subqueries); if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 2430976825f..52c62cc4cec 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -111,7 +111,7 @@ public: const StorageMetadataPtr & metadata_snapshot = {}, bool allow_aggregations = false, bool allow_self_aliases = true, - bool dry_run = false) const; + bool execute_scalar_subqueries = true) const; /// Analyze and rewrite select query TreeRewriterResultPtr analyzeSelect( diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference index 6ed281c757a..98fb6a68656 100644 --- a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference @@ -1,2 +1,4 @@ 1 1 +1 +1 diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql index 5fde10d5d09..32b757f54a3 100644 --- a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql @@ -15,3 +15,20 @@ SELECT sum(x) == 42 FROM foo; ALTER TABLE foo UPDATE x = 1 WHERE x IN (SELECT x FROM foo WHERE x != 0); SELECT sum(x) == 9 FROM foo; + +DROP TABLE IF EXISTS bar; + +CREATE TABLE bar (ts DateTime, x UInt64) +ENGINE = Memory; + +INSERT INTO bar (ts, x) SELECT toDateTime('2020-01-01 00:05:00'), number from system.numbers_mt LIMIT 10; + +SET mutations_sync = 1; + +ALTER TABLE bar UPDATE x = 1 WHERE x = (SELECT x from bar WHERE x = 4); + +SELECT sum(x) == 42 FROM bar; + +ALTER TABLE bar UPDATE x = 1 WHERE x IN (SELECT x FROM bar WHERE x != 0); + +SELECT sum(x) == 9 FROM bar; From a874ca29edfdaf5e39efb19ec0d3743441525661 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 21 Oct 2021 09:31:38 -0600 Subject: [PATCH 903/950] Change link in nav from community -> learn (#30508) --- website/templates/global/nav.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/global/nav.html b/website/templates/global/nav.html index 7e120c99126..d775c88e4a5 100644 --- a/website/templates/global/nav.html +++ b/website/templates/global/nav.html @@ -42,7 +42,7 @@ Documentation