diff --git a/docker/packager/binary-builder/build.sh b/docker/packager/binary-builder/build.sh index 032aceb0af3..bd5f2fe8466 100755 --- a/docker/packager/binary-builder/build.sh +++ b/docker/packager/binary-builder/build.sh @@ -111,6 +111,7 @@ fi mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output [ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /output [ -x ./programs/self-extracting/clickhouse-stripped ] && mv ./programs/self-extracting/clickhouse-stripped /output +[ -x ./programs/self-extracting/clickhouse-keeper ] && mv ./programs/self-extracting/clickhouse-keeper /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds mv ./programs/*.dict ./programs/*.options ./programs/*_seed_corpus.zip /output ||: # libFuzzer oss-fuzz compatible infrastructure diff --git a/programs/self-extracting/CMakeLists.txt b/programs/self-extracting/CMakeLists.txt index 4b6dd07f618..32b686d40dd 100644 --- a/programs/self-extracting/CMakeLists.txt +++ b/programs/self-extracting/CMakeLists.txt @@ -10,9 +10,24 @@ else () set (COMPRESSOR "${PROJECT_BINARY_DIR}/utils/self-extracting-executable/compressor") endif () -add_custom_target (self-extracting ALL +add_custom_target (self-extracting-server ALL ${CMAKE_COMMAND} -E remove clickhouse clickhouse-stripped COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse ../clickhouse COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse-stripped ../clickhouse-stripped DEPENDS clickhouse clickhouse-stripped compressor ) + +set(self_extracting_deps "self-extracting-server") + +if (BUILD_STANDALONE_KEEPER) + add_custom_target (self-extracting-keeper ALL + ${CMAKE_COMMAND} -E remove clickhouse-keeper + COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse-keeper ../clickhouse-keeper + DEPENDS compressor clickhouse-keeper + ) + list(APPEND self_extracting_deps "self-extracting-keeper") +endif() + +add_custom_target (self-extracting ALL + DEPENDS ${self_extracting_deps} +) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 68e0131c91b..2e7aa0d086f 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -185,7 +185,6 @@ void registerCodecDeflateQpl(CompressionCodecFactory & factory); /// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD void registerCodecDelta(CompressionCodecFactory & factory); void registerCodecT64(CompressionCodecFactory & factory); void registerCodecDoubleDelta(CompressionCodecFactory & factory); @@ -193,7 +192,6 @@ void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecEncrypted(CompressionCodecFactory & factory); void registerCodecFPC(CompressionCodecFactory & factory); void registerCodecGCD(CompressionCodecFactory & factory); -#endif CompressionCodecFactory::CompressionCodecFactory() { @@ -205,7 +203,6 @@ CompressionCodecFactory::CompressionCodecFactory() #endif registerCodecLZ4HC(*this); registerCodecMultiple(*this); -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD registerCodecDelta(*this); registerCodecT64(*this); registerCodecDoubleDelta(*this); @@ -216,7 +213,6 @@ CompressionCodecFactory::CompressionCodecFactory() registerCodecDeflateQpl(*this); #endif registerCodecGCD(*this); -#endif default_codec = get("LZ4", {}); } diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp deleted file mode 100644 index 2017adcc58d..00000000000 --- a/src/Coordination/Standalone/Context.cpp +++ /dev/null @@ -1,486 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include - -#include - -#include - -#include -#include - -namespace ProfileEvents -{ - extern const Event ContextLock; - extern const Event ContextLockWaitMicroseconds; -} - -namespace CurrentMetrics -{ - extern const Metric ContextLockWait; - extern const Metric BackgroundSchedulePoolTask; - extern const Metric BackgroundSchedulePoolSize; - extern const Metric IOWriterThreads; - extern const Metric IOWriterThreadsActive; - extern const Metric IOWriterThreadsScheduled; -} - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int UNSUPPORTED_METHOD; -} - -struct ContextSharedPart : boost::noncopyable -{ - ContextSharedPart() - : macros(std::make_unique()) - {} - - ~ContextSharedPart() - { - if (keeper_dispatcher) - { - try - { - keeper_dispatcher->shutdown(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - /// Wait for thread pool for background reads and writes, - /// since it may use per-user MemoryTracker which will be destroyed here. - if (asynchronous_remote_fs_reader) - { - try - { - asynchronous_remote_fs_reader->wait(); - asynchronous_remote_fs_reader.reset(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - if (asynchronous_local_fs_reader) - { - try - { - asynchronous_local_fs_reader->wait(); - asynchronous_local_fs_reader.reset(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - if (synchronous_local_fs_reader) - { - try - { - synchronous_local_fs_reader->wait(); - synchronous_local_fs_reader.reset(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - if (threadpool_writer) - { - try - { - threadpool_writer->wait(); - threadpool_writer.reset(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - } - - /// For access of most of shared objects. - mutable SharedMutex mutex; - - ServerSettings server_settings; - - String path; /// Path to the data directory, with a slash at the end. - ConfigurationPtr config; /// Global configuration settings. - MultiVersion macros; /// Substitutions extracted from config. - OnceFlag schedule_pool_initialized; - mutable std::unique_ptr schedule_pool; /// A thread pool that can run different jobs in background - RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml - - mutable OnceFlag readers_initialized; - mutable std::unique_ptr asynchronous_remote_fs_reader; - mutable std::unique_ptr asynchronous_local_fs_reader; - mutable std::unique_ptr synchronous_local_fs_reader; - -#if USE_LIBURING - mutable OnceFlag io_uring_reader_initialized; - mutable std::unique_ptr io_uring_reader; -#endif - - mutable OnceFlag threadpool_writer_initialized; - mutable std::unique_ptr threadpool_writer; - - mutable ThrottlerPtr remote_read_throttler; /// A server-wide throttler for remote IO reads - mutable ThrottlerPtr remote_write_throttler; /// A server-wide throttler for remote IO writes - - mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads - mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes - - std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage - - mutable std::mutex keeper_dispatcher_mutex; - mutable std::shared_ptr keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex); -}; - -ContextData::ContextData() = default; -ContextData::ContextData(const ContextData &) = default; - -Context::Context() = default; -Context::Context(const Context & rhs) : ContextData(rhs), std::enable_shared_from_this(rhs) {} -Context::~Context() = default; - -SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default; -SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) noexcept = default; -SharedContextHolder::SharedContextHolder() = default; -SharedContextHolder::~SharedContextHolder() = default; -SharedContextHolder::SharedContextHolder(std::unique_ptr shared_context) - : shared(std::move(shared_context)) {} - -void SharedContextHolder::reset() { shared.reset(); } - -void Context::makeGlobalContext() -{ - initGlobal(); - global_context = shared_from_this(); -} - -ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) -{ - auto res = std::shared_ptr(new Context); - res->shared = shared_part; - return res; -} - -void Context::initGlobal() -{ - assert(!global_context_instance); - global_context_instance = shared_from_this(); -} - -SharedContextHolder Context::createShared() -{ - return SharedContextHolder(std::make_unique()); -} - - -ContextMutablePtr Context::getGlobalContext() const -{ - auto ptr = global_context.lock(); - if (!ptr) throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no global context or global context has expired"); - return ptr; -} - -std::unique_lock Context::getGlobalLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::unique_lock(shared->mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -std::shared_lock Context::getGlobalSharedLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::shared_lock(shared->mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -std::unique_lock Context::getLocalLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::unique_lock(mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -std::shared_lock Context::getLocalSharedLock() const -{ - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; - Stopwatch watch; - auto lock = std::shared_lock(mutex); - ProfileEvents::increment(ProfileEvents::ContextLockWaitMicroseconds, watch.elapsedMicroseconds()); - return lock; -} - -String Context::getPath() const -{ - auto lock = getGlobalSharedLock(); - return shared->path; -} - -void Context::setPath(const String & path) -{ - auto lock = getGlobalLock(); - shared->path = path; -} - -MultiVersion::Version Context::getMacros() const -{ - return shared->macros.get(); -} - -void Context::setMacros(std::unique_ptr && macros) -{ - shared->macros.set(std::move(macros)); -} - -BackgroundSchedulePool & Context::getSchedulePool() const -{ - callOnce(shared->schedule_pool_initialized, [&] { - shared->schedule_pool = std::make_unique( - shared->server_settings.background_schedule_pool_size, - CurrentMetrics::BackgroundSchedulePoolTask, - CurrentMetrics::BackgroundSchedulePoolSize, - "BgSchPool"); - }); - - return *shared->schedule_pool; -} - -void Context::setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config) -{ - shared->remote_host_filter.setValuesFromConfig(config); -} - -const RemoteHostFilter & Context::getRemoteHostFilter() const -{ - return shared->remote_host_filter; -} - -IAsynchronousReader & Context::getThreadPoolReader(FilesystemReaderType type) const -{ - callOnce(shared->readers_initialized, [&] { - const auto & config = getConfigRef(); - shared->asynchronous_remote_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER, config); - shared->asynchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER, config); - shared->synchronous_local_fs_reader = createThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER, config); - }); - - switch (type) - { - case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER: - return *shared->asynchronous_remote_fs_reader; - case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER: - return *shared->asynchronous_local_fs_reader; - case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER: - return *shared->synchronous_local_fs_reader; - } -} - -#if USE_LIBURING -IOUringReader & Context::getIOUringReader() const -{ - callOnce(shared->io_uring_reader_initialized, [&] { - shared->io_uring_reader = createIOUringReader(); - }); - - return *shared->io_uring_reader; -} -#endif - -std::shared_ptr Context::getFilesystemCacheLog() const -{ - return nullptr; -} - -std::shared_ptr Context::getFilesystemReadPrefetchesLog() const -{ - return nullptr; -} - -std::shared_ptr Context::getBlobStorageLog() const -{ - return nullptr; -} - -void Context::setConfig(const ConfigurationPtr & config) -{ - auto lock = getGlobalLock(); - shared->config = config; -} - -const Poco::Util::AbstractConfiguration & Context::getConfigRef() const -{ - auto lock = getGlobalSharedLock(); - return shared->config ? *shared->config : Poco::Util::Application::instance().config(); -} - -std::shared_ptr Context::getAsyncReadCounters() const -{ - auto lock = getLocalLock(); - if (!async_read_counters) - async_read_counters = std::make_shared(); - return async_read_counters; -} - -ThreadPool & Context::getThreadPoolWriter() const -{ - callOnce(shared->threadpool_writer_initialized, [&] { - const auto & config = getConfigRef(); - auto pool_size = config.getUInt(".threadpool_writer_pool_size", 100); - auto queue_size = config.getUInt(".threadpool_writer_queue_size", 1000000); - - shared->threadpool_writer = std::make_unique( - CurrentMetrics::IOWriterThreads, CurrentMetrics::IOWriterThreadsActive, CurrentMetrics::IOWriterThreadsScheduled, pool_size, pool_size, queue_size); - }); - - return *shared->threadpool_writer; -} - -ThrottlerPtr Context::getRemoteReadThrottler() const -{ - return nullptr; -} - -ThrottlerPtr Context::getRemoteWriteThrottler() const -{ - return nullptr; -} - -ThrottlerPtr Context::getLocalReadThrottler() const -{ - return nullptr; -} - -ThrottlerPtr Context::getLocalWriteThrottler() const -{ - return nullptr; -} - -ReadSettings Context::getReadSettings() const -{ - return ReadSettings{}; -} - -ResourceManagerPtr Context::getResourceManager() const -{ - return nullptr; -} - -ClassifierPtr Context::getWorkloadClassifier() const -{ - return nullptr; -} - -void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const -{ - const auto & config_ref = getConfigRef(); - - std::lock_guard lock(shared->keeper_dispatcher_mutex); - - if (shared->keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); - - if (config_ref.has("keeper_server")) - { - shared->keeper_dispatcher = std::make_shared(); - shared->keeper_dispatcher->initialize(config_ref, true, start_async, getMacros()); - } -} - -std::shared_ptr Context::getKeeperDispatcher() const -{ - std::lock_guard lock(shared->keeper_dispatcher_mutex); - if (!shared->keeper_dispatcher) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests"); - - return shared->keeper_dispatcher; -} - -std::shared_ptr Context::tryGetKeeperDispatcher() const -{ - std::lock_guard lock(shared->keeper_dispatcher_mutex); - return shared->keeper_dispatcher; -} - -void Context::shutdownKeeperDispatcher() const -{ - std::lock_guard lock(shared->keeper_dispatcher_mutex); - if (shared->keeper_dispatcher) - { - shared->keeper_dispatcher->shutdown(); - shared->keeper_dispatcher.reset(); - } -} - -void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config_) -{ - std::lock_guard lock(shared->keeper_dispatcher_mutex); - if (!shared->keeper_dispatcher) - return; - - shared->keeper_dispatcher->updateConfiguration(config_, getMacros()); -} - -std::shared_ptr Context::getZooKeeper() const -{ - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot connect to ZooKeeper from Keeper"); -} - -const S3SettingsByEndpoint & Context::getStorageS3Settings() const -{ - std::lock_guard lock(shared->mutex); - - if (!shared->storage_s3_settings) - { - const auto & config = shared->config ? *shared->config : Poco::Util::Application::instance().config(); - shared->storage_s3_settings.emplace().loadFromConfig(config, "s3", getSettingsRef()); - } - - return *shared->storage_s3_settings; -} - -const ServerSettings & Context::getServerSettings() const -{ - return shared->server_settings; -} - -bool Context::hasTraceCollector() const -{ - return false; -} - -bool Context::isBackgroundOperationContext() const -{ - return false; -} - -} diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h deleted file mode 100644 index d3bbfececed..00000000000 --- a/src/Coordination/Standalone/Context.h +++ /dev/null @@ -1,178 +0,0 @@ -#pragma once - -#include - -#include - -#include -#include -#include - -#include - -#include -#include -#include - -#include -#include - -#include - -#include - -#include "config.h" -namespace zkutil -{ - class ZooKeeper; - using ZooKeeperPtr = std::shared_ptr; -} - -namespace DB -{ - -struct ContextSharedPart; -class Macros; -class FilesystemCacheLog; -class FilesystemReadPrefetchesLog; -class BlobStorageLog; -class IOUringReader; -class S3SettingsByEndpoint; - -/// A small class which owns ContextShared. -/// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. -struct SharedContextHolder -{ - ~SharedContextHolder(); - SharedContextHolder(); - explicit SharedContextHolder(std::unique_ptr shared_context); - SharedContextHolder(SharedContextHolder &&) noexcept; - - SharedContextHolder & operator=(SharedContextHolder &&) noexcept; - - ContextSharedPart * get() const { return shared.get(); } - void reset(); -private: - std::unique_ptr shared; -}; - -class ContextData -{ -protected: - ContextWeakMutablePtr global_context; - inline static ContextPtr global_context_instance; - ContextSharedPart * shared; - - /// Query metrics for reading data asynchronously with IAsynchronousReader. - mutable std::shared_ptr async_read_counters; - - Settings settings; /// Setting for query execution. - -public: - /// Use copy constructor or createGlobal() instead - ContextData(); - ContextData(const ContextData &); -}; - -class Context : public ContextData, public std::enable_shared_from_this -{ -private: - /// ContextData mutex - mutable SharedMutex mutex; - - Context(); - Context(const Context &); - - std::unique_lock getGlobalLock() const; - - std::shared_lock getGlobalSharedLock() const; - - std::unique_lock getLocalLock() const; - - std::shared_lock getLocalSharedLock() const; - -public: - /// Create initial Context with ContextShared and etc. - static ContextMutablePtr createGlobal(ContextSharedPart * shared_part); - static SharedContextHolder createShared(); - - ContextMutablePtr getGlobalContext() const; - static ContextPtr getGlobalContextInstance() { return global_context_instance; } - - void makeGlobalContext(); - void initGlobal(); - - ~Context(); - - using ConfigurationPtr = Poco::AutoPtr; - - /// Global application configuration settings. - void setConfig(const ConfigurationPtr & config); - const Poco::Util::AbstractConfiguration & getConfigRef() const; - - const Settings & getSettingsRef() const { return settings; } - - String getPath() const; - void setPath(const String & path); - - MultiVersion::Version getMacros() const; - void setMacros(std::unique_ptr && macros); - - BackgroundSchedulePool & getSchedulePool() const; - - /// Storage of allowed hosts from config.xml - void setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config); - const RemoteHostFilter & getRemoteHostFilter() const; - - std::shared_ptr getFilesystemCacheLog() const; - std::shared_ptr getFilesystemReadPrefetchesLog() const; - std::shared_ptr getBlobStorageLog() const; - - enum class ApplicationType : uint8_t - { - KEEPER, - SERVER, - }; - - void setApplicationType(ApplicationType) {} - ApplicationType getApplicationType() const { return ApplicationType::KEEPER; } - - IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const; -#if USE_LIBURING - IOUringReader & getIOUringReader() const; -#endif - std::shared_ptr getAsyncReadCounters() const; - ThreadPool & getThreadPoolWriter() const; - - ThrottlerPtr getRemoteReadThrottler() const; - ThrottlerPtr getRemoteWriteThrottler() const; - - ThrottlerPtr getLocalReadThrottler() const; - ThrottlerPtr getLocalWriteThrottler() const; - - ReadSettings getReadSettings() const; - - /// Resource management related - ResourceManagerPtr getResourceManager() const; - ClassifierPtr getWorkloadClassifier() const; - - std::shared_ptr getKeeperDispatcher() const; - std::shared_ptr tryGetKeeperDispatcher() const; - void initializeKeeperDispatcher(bool start_async) const; - void shutdownKeeperDispatcher() const; - void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); - - zkutil::ZooKeeperPtr getZooKeeper() const; - - const S3SettingsByEndpoint & getStorageS3Settings() const; - - const String & getUserName() const { static std::string user; return user; } - - const ServerSettings & getServerSettings() const; - - bool hasTraceCollector() const; - - bool isBackgroundOperationContext() const; -}; - -} diff --git a/src/Coordination/Standalone/Settings.cpp b/src/Coordination/Standalone/Settings.cpp deleted file mode 100644 index 12a7a42ffac..00000000000 --- a/src/Coordination/Standalone/Settings.cpp +++ /dev/null @@ -1,24 +0,0 @@ -#include - -namespace DB -{ - -IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS) - -std::vector Settings::getAllRegisteredNames() const -{ - std::vector all_settings; - for (const auto & setting_field : all()) - { - all_settings.push_back(setting_field.getName()); - } - return all_settings; -} - -void Settings::set(std::string_view name, const Field & value) -{ - BaseSettings::set(name, value); -} - - -} diff --git a/src/Coordination/Standalone/ThreadStatusExt.cpp b/src/Coordination/Standalone/ThreadStatusExt.cpp deleted file mode 100644 index fc78233d9dc..00000000000 --- a/src/Coordination/Standalone/ThreadStatusExt.cpp +++ /dev/null @@ -1,19 +0,0 @@ -#include -#include - -namespace DB -{ - -void CurrentThread::detachFromGroupIfNotDetached() -{ -} - -void CurrentThread::attachToGroup(const ThreadGroupPtr &) -{ -} - -void ThreadStatus::initGlobalProfiler(UInt64 /*global_profiler_real_time_period*/, UInt64 /*global_profiler_cpu_time_period*/) -{ -} - -} diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index caa8b3fdffd..7d094e2a107 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -380,15 +380,6 @@ void SettingFieldString::readBinary(ReadBuffer & in) *this = std::move(str); } -/// Unbeautiful workaround for clickhouse-keeper standalone build ("-DBUILD_STANDALONE_KEEPER=1"). -/// In this build, we don't build and link library dbms (to which SettingsField.cpp belongs) but -/// only build SettingsField.cpp. Further dependencies, e.g. DataTypeString and DataTypeMap below, -/// require building of further files for clickhouse-keeper. To keep dependencies slim, we don't do -/// that. The linker does not complain only because clickhouse-keeper does not call any of below -/// functions. A cleaner alternative would be more modular libraries, e.g. one for data types, which -/// could then be linked by the server and the linker. -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD - SettingFieldMap::SettingFieldMap(const Field & f) : value(fieldToMap(f)) {} String SettingFieldMap::toString() const @@ -428,42 +419,6 @@ void SettingFieldMap::readBinary(ReadBuffer & in) *this = map; } -#else - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -SettingFieldMap::SettingFieldMap(const Field &) : value(Map()) {} -String SettingFieldMap::toString() const -{ - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); -} - - -SettingFieldMap & SettingFieldMap::operator =(const Field &) -{ - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); -} - -void SettingFieldMap::parseFromString(const String &) -{ - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); -} - -void SettingFieldMap::writeBinary(WriteBuffer &) const -{ - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); -} - -void SettingFieldMap::readBinary(ReadBuffer &) -{ - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Setting of type Map not supported"); -} - -#endif - namespace { char stringToChar(const String & str) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 19809348921..266141815e3 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -247,12 +247,6 @@ struct SettingFieldString void readBinary(ReadBuffer & in); }; -#ifdef CLICKHOUSE_KEEPER_STANDALONE_BUILD -#define NORETURN [[noreturn]] -#else -#define NORETURN -#endif - struct SettingFieldMap { public: @@ -269,11 +263,11 @@ public: operator const Map &() const { return value; } /// NOLINT explicit operator Field() const { return value; } - NORETURN String toString() const; - NORETURN void parseFromString(const String & str); + String toString() const; + void parseFromString(const String & str); - NORETURN void writeBinary(WriteBuffer & out) const; - NORETURN void readBinary(ReadBuffer & in); + void writeBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); }; #undef NORETURN diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index b2c425ceb79..48f76769a09 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -502,9 +502,7 @@ private: if (collectCrashLog) collectCrashLog(sig, thread_num, query_id, stack_trace); -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD Context::getGlobalContextInstance()->handleCrash(); -#endif /// Send crash report to developers (if configured) if (sig != SanitizerTrap) @@ -533,8 +531,6 @@ private: } } - /// ClickHouse Keeper does not link to some parts of Settings. -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD /// List changed settings. if (!query_id.empty()) { @@ -549,7 +545,6 @@ private: LOG_FATAL(log, "Changed settings: {}", changed_settings); } } -#endif /// When everything is done, we will try to send these error messages to the client. if (thread_ptr) diff --git a/src/Daemon/SentryWriter.cpp b/src/Daemon/SentryWriter.cpp index 9479dd65730..c51a1100639 100644 --- a/src/Daemon/SentryWriter.cpp +++ b/src/Daemon/SentryWriter.cpp @@ -19,7 +19,7 @@ #include "config.h" #include -#if USE_SENTRY && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_SENTRY # include # include diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index c77709c27eb..bb9761a3905 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -78,7 +78,6 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c std::unique_ptr buf; -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD if (with_file_cache) { auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path); @@ -96,7 +95,6 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c /* read_until_position */std::nullopt, cache_log); } -#endif /// Can't wrap CachedOnDiskReadBufferFromFile in CachedInMemoryReadBufferFromFile because the /// former doesn't support seeks. diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 59cc82d8c81..5c45a258806 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -195,7 +195,6 @@ public: /// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage))) String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); } -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD /// Add a cache layer. /// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage)) /// There can be any number of cache layers: @@ -204,7 +203,6 @@ public: /// Get names of all cache layers. Name is how cache is defined in configuration file. NameSet getCacheLayersNames() const override; -#endif bool supportsStat() const override { return metadata_storage->supportsStat(); } struct stat stat(const String & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index 44854633d65..56d5d11ef8a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -222,11 +222,7 @@ ObjectKeyWithMetadata DiskObjectStorageMetadata::popLastObject() bool DiskObjectStorageMetadata::getWriteFullObjectKeySetting() { -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD return Context::getGlobalContextInstance()->getServerSettings().storage_metadata_write_full_object_key; -#else - return false; -#endif } } diff --git a/src/Disks/ObjectStorages/MetadataStorageFactory.cpp b/src/Disks/ObjectStorages/MetadataStorageFactory.cpp index ab7c2069b43..a690ecd2757 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFactory.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFactory.cpp @@ -2,9 +2,7 @@ #include #include #include -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include -#endif #include #include @@ -135,7 +133,6 @@ void registerPlainRewritableMetadataStorage(MetadataStorageFactory & factory) }); } -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD void registerMetadataStorageFromStaticFilesWebServer(MetadataStorageFactory & factory) { factory.registerMetadataStorageType("web", []( @@ -147,7 +144,6 @@ void registerMetadataStorageFromStaticFilesWebServer(MetadataStorageFactory & fa return std::make_shared(assert_cast(*object_storage)); }); } -#endif void registerMetadataStorages() { @@ -155,9 +151,7 @@ void registerMetadataStorages() registerMetadataStorageFromDisk(factory); registerPlainMetadataStorage(factory); registerPlainRewritableMetadataStorage(factory); -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD registerMetadataStorageFromStaticFilesWebServer(factory); -#endif } } diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 1bf8250adff..5698d2ad588 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -7,19 +7,17 @@ #include #include #endif -#if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_HDFS #include #include #endif -#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_AZURE_BLOB_STORAGE #include #include #endif -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD #include #include #include -#endif #include #include #include @@ -284,7 +282,7 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory) #endif -#if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_HDFS void registerHDFSObjectStorage(ObjectStorageFactory & factory) { factory.registerObjectStorageType( @@ -309,7 +307,7 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) } #endif -#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_AZURE_BLOB_STORAGE void registerAzureObjectStorage(ObjectStorageFactory & factory) { auto creator = []( @@ -333,7 +331,6 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory) } #endif -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD void registerWebObjectStorage(ObjectStorageFactory & factory) { factory.registerObjectStorageType("web", []( @@ -381,7 +378,6 @@ void registerLocalObjectStorage(ObjectStorageFactory & factory) factory.registerObjectStorageType("local_blob_storage", creator); factory.registerObjectStorageType("local", creator); } -#endif void registerObjectStorages() { @@ -393,18 +389,16 @@ void registerObjectStorages() registerS3PlainRewritableObjectStorage(factory); #endif -#if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_HDFS registerHDFSObjectStorage(factory); #endif -#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_AZURE_BLOB_STORAGE registerAzureObjectStorage(factory); #endif -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD registerWebObjectStorage(factory); registerLocalObjectStorage(factory); -#endif } } diff --git a/src/Disks/ObjectStorages/createMetadataStorageMetrics.h b/src/Disks/ObjectStorages/createMetadataStorageMetrics.h index 6dddc227ade..5cf1fbef2ab 100644 --- a/src/Disks/ObjectStorages/createMetadataStorageMetrics.h +++ b/src/Disks/ObjectStorages/createMetadataStorageMetrics.h @@ -1,14 +1,14 @@ #pragma once +#include "config.h" + #if USE_AWS_S3 # include #endif -#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_AZURE_BLOB_STORAGE # include #endif -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD -# include -#endif +#include #include namespace ProfileEvents @@ -42,7 +42,7 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create inline MetadataStorageMetrics MetadataStorageMetrics::create() { @@ -53,7 +53,6 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create inline MetadataStorageMetrics MetadataStorageMetrics::create() { @@ -62,6 +61,5 @@ inline MetadataStorageMetrics MetadataStorageMetrics::creategetBlobStorageLog()) { auto log_writer = std::make_shared(std::move(blob_storage_log)); @@ -67,7 +66,6 @@ BlobStorageLogWriterPtr BlobStorageLogWriter::create(const String & disk_name) return log_writer; } -#endif return {}; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f9b91a45978..d3f152b7a67 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,7 +1,5 @@ #pragma once -#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD - #include #include #include @@ -1451,9 +1449,3 @@ struct HTTPContext : public IHTTPContext }; } - -#else - -#include - -#endif diff --git a/src/Server/PrometheusRequestHandler.cpp b/src/Server/PrometheusRequestHandler.cpp index dff960f7031..1f3e038a1f5 100644 --- a/src/Server/PrometheusRequestHandler.cpp +++ b/src/Server/PrometheusRequestHandler.cpp @@ -18,9 +18,6 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe { try { - /// Raw config reference is used here to avoid dependency on Context and ServerSettings. - /// This is painful, because this class is also used in a build with CLICKHOUSE_KEEPER_STANDALONE_BUILD=1 - /// And there ordinary Context is replaced with a tiny clone. const auto & config = server.config(); unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT); diff --git a/src/Server/ProtocolServerAdapter.cpp b/src/Server/ProtocolServerAdapter.cpp index 8d14a849894..b41ad2376f1 100644 --- a/src/Server/ProtocolServerAdapter.cpp +++ b/src/Server/ProtocolServerAdapter.cpp @@ -1,7 +1,7 @@ #include #include -#if USE_GRPC && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_GRPC #include #endif @@ -37,7 +37,7 @@ ProtocolServerAdapter::ProtocolServerAdapter( { } -#if USE_GRPC && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_GRPC class ProtocolServerAdapter::GRPCServerAdapterImpl : public Impl { public: diff --git a/src/Server/ProtocolServerAdapter.h b/src/Server/ProtocolServerAdapter.h index dd11c1dfc58..76a6776ed9c 100644 --- a/src/Server/ProtocolServerAdapter.h +++ b/src/Server/ProtocolServerAdapter.h @@ -23,7 +23,7 @@ public: ProtocolServerAdapter & operator =(ProtocolServerAdapter && src) = default; ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr tcp_server_); -#if USE_GRPC && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) +#if USE_GRPC ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr grpc_server_); #endif