mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Remove Keeper standalone build
This commit is contained in:
parent
6b47171f2c
commit
4a9daa202d
@ -111,6 +111,7 @@ fi
|
|||||||
mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output
|
mv ./programs/clickhouse* /output || mv ./programs/*_fuzzer /output
|
||||||
[ -x ./programs/self-extracting/clickhouse ] && mv ./programs/self-extracting/clickhouse /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-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 ./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
|
mv ./programs/*.dict ./programs/*.options ./programs/*_seed_corpus.zip /output ||: # libFuzzer oss-fuzz compatible infrastructure
|
||||||
|
|
||||||
|
@ -10,9 +10,24 @@ else ()
|
|||||||
set (COMPRESSOR "${PROJECT_BINARY_DIR}/utils/self-extracting-executable/compressor")
|
set (COMPRESSOR "${PROJECT_BINARY_DIR}/utils/self-extracting-executable/compressor")
|
||||||
endif ()
|
endif ()
|
||||||
|
|
||||||
add_custom_target (self-extracting ALL
|
add_custom_target (self-extracting-server ALL
|
||||||
${CMAKE_COMMAND} -E remove clickhouse clickhouse-stripped
|
${CMAKE_COMMAND} -E remove clickhouse clickhouse-stripped
|
||||||
COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse ../clickhouse
|
COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse ../clickhouse
|
||||||
COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse-stripped ../clickhouse-stripped
|
COMMAND ${COMPRESSOR} ${DECOMPRESSOR} clickhouse-stripped ../clickhouse-stripped
|
||||||
DEPENDS clickhouse clickhouse-stripped compressor
|
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}
|
||||||
|
)
|
||||||
|
@ -185,7 +185,6 @@ void registerCodecDeflateQpl(CompressionCodecFactory & factory);
|
|||||||
|
|
||||||
/// Keeper use only general-purpose codecs, so we don't need these special codecs
|
/// Keeper use only general-purpose codecs, so we don't need these special codecs
|
||||||
/// in standalone build
|
/// in standalone build
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
void registerCodecDelta(CompressionCodecFactory & factory);
|
void registerCodecDelta(CompressionCodecFactory & factory);
|
||||||
void registerCodecT64(CompressionCodecFactory & factory);
|
void registerCodecT64(CompressionCodecFactory & factory);
|
||||||
void registerCodecDoubleDelta(CompressionCodecFactory & factory);
|
void registerCodecDoubleDelta(CompressionCodecFactory & factory);
|
||||||
@ -193,7 +192,6 @@ void registerCodecGorilla(CompressionCodecFactory & factory);
|
|||||||
void registerCodecEncrypted(CompressionCodecFactory & factory);
|
void registerCodecEncrypted(CompressionCodecFactory & factory);
|
||||||
void registerCodecFPC(CompressionCodecFactory & factory);
|
void registerCodecFPC(CompressionCodecFactory & factory);
|
||||||
void registerCodecGCD(CompressionCodecFactory & factory);
|
void registerCodecGCD(CompressionCodecFactory & factory);
|
||||||
#endif
|
|
||||||
|
|
||||||
CompressionCodecFactory::CompressionCodecFactory()
|
CompressionCodecFactory::CompressionCodecFactory()
|
||||||
{
|
{
|
||||||
@ -205,7 +203,6 @@ CompressionCodecFactory::CompressionCodecFactory()
|
|||||||
#endif
|
#endif
|
||||||
registerCodecLZ4HC(*this);
|
registerCodecLZ4HC(*this);
|
||||||
registerCodecMultiple(*this);
|
registerCodecMultiple(*this);
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
registerCodecDelta(*this);
|
registerCodecDelta(*this);
|
||||||
registerCodecT64(*this);
|
registerCodecT64(*this);
|
||||||
registerCodecDoubleDelta(*this);
|
registerCodecDoubleDelta(*this);
|
||||||
@ -216,7 +213,6 @@ CompressionCodecFactory::CompressionCodecFactory()
|
|||||||
registerCodecDeflateQpl(*this);
|
registerCodecDeflateQpl(*this);
|
||||||
#endif
|
#endif
|
||||||
registerCodecGCD(*this);
|
registerCodecGCD(*this);
|
||||||
#endif
|
|
||||||
|
|
||||||
default_codec = get("LZ4", {});
|
default_codec = get("LZ4", {});
|
||||||
}
|
}
|
||||||
|
@ -1,486 +0,0 @@
|
|||||||
#include <Interpreters/Context.h>
|
|
||||||
|
|
||||||
#include <Common/Config/ConfigProcessor.h>
|
|
||||||
#include <Common/Macros.h>
|
|
||||||
#include <Common/ThreadPool.h>
|
|
||||||
#include <Common/callOnce.h>
|
|
||||||
#include <Disks/IO/IOUringReader.h>
|
|
||||||
#include <IO/S3Settings.h>
|
|
||||||
#include <Disks/IO/getIOUringReader.h>
|
|
||||||
|
|
||||||
#include <Core/ServerSettings.h>
|
|
||||||
|
|
||||||
#include <boost/noncopyable.hpp>
|
|
||||||
|
|
||||||
#include <memory>
|
|
||||||
#include <cassert>
|
|
||||||
|
|
||||||
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<Macros>())
|
|
||||||
{}
|
|
||||||
|
|
||||||
~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> macros; /// Substitutions extracted from config.
|
|
||||||
OnceFlag schedule_pool_initialized;
|
|
||||||
mutable std::unique_ptr<BackgroundSchedulePool> 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<IAsynchronousReader> asynchronous_remote_fs_reader;
|
|
||||||
mutable std::unique_ptr<IAsynchronousReader> asynchronous_local_fs_reader;
|
|
||||||
mutable std::unique_ptr<IAsynchronousReader> synchronous_local_fs_reader;
|
|
||||||
|
|
||||||
#if USE_LIBURING
|
|
||||||
mutable OnceFlag io_uring_reader_initialized;
|
|
||||||
mutable std::unique_ptr<IOUringReader> io_uring_reader;
|
|
||||||
#endif
|
|
||||||
|
|
||||||
mutable OnceFlag threadpool_writer_initialized;
|
|
||||||
mutable std::unique_ptr<ThreadPool> 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<S3SettingsByEndpoint> storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage
|
|
||||||
|
|
||||||
mutable std::mutex keeper_dispatcher_mutex;
|
|
||||||
mutable std::shared_ptr<KeeperDispatcher> 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<Context>(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<ContextSharedPart> 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<Context>(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<ContextSharedPart>());
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
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<SharedMutex> 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<SharedMutex> 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<SharedMutex> 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<SharedMutex> 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<Macros>::Version Context::getMacros() const
|
|
||||||
{
|
|
||||||
return shared->macros.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
void Context::setMacros(std::unique_ptr<Macros> && macros)
|
|
||||||
{
|
|
||||||
shared->macros.set(std::move(macros));
|
|
||||||
}
|
|
||||||
|
|
||||||
BackgroundSchedulePool & Context::getSchedulePool() const
|
|
||||||
{
|
|
||||||
callOnce(shared->schedule_pool_initialized, [&] {
|
|
||||||
shared->schedule_pool = std::make_unique<BackgroundSchedulePool>(
|
|
||||||
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<FilesystemCacheLog> Context::getFilesystemCacheLog() const
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::shared_ptr<FilesystemReadPrefetchesLog> Context::getFilesystemReadPrefetchesLog() const
|
|
||||||
{
|
|
||||||
return nullptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
std::shared_ptr<BlobStorageLog> 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<AsyncReadCounters> Context::getAsyncReadCounters() const
|
|
||||||
{
|
|
||||||
auto lock = getLocalLock();
|
|
||||||
if (!async_read_counters)
|
|
||||||
async_read_counters = std::make_shared<AsyncReadCounters>();
|
|
||||||
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<ThreadPool>(
|
|
||||||
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<KeeperDispatcher>();
|
|
||||||
shared->keeper_dispatcher->initialize(config_ref, true, start_async, getMacros());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
std::shared_ptr<KeeperDispatcher> 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<KeeperDispatcher> 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<zkutil::ZooKeeper> 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -1,178 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <Interpreters/Context_fwd.h>
|
|
||||||
|
|
||||||
#include <Coordination/KeeperDispatcher.h>
|
|
||||||
|
|
||||||
#include <Common/MultiVersion.h>
|
|
||||||
#include <Common/RemoteHostFilter.h>
|
|
||||||
#include <Common/SharedMutex.h>
|
|
||||||
|
|
||||||
#include <Disks/IO/getThreadPoolReader.h>
|
|
||||||
|
|
||||||
#include <Core/Settings.h>
|
|
||||||
#include <Core/ServerSettings.h>
|
|
||||||
#include <Core/BackgroundSchedulePool.h>
|
|
||||||
|
|
||||||
#include <IO/AsyncReadCounters.h>
|
|
||||||
#include <Common/Scheduler/IResourceManager.h>
|
|
||||||
|
|
||||||
#include <Poco/Util/Application.h>
|
|
||||||
|
|
||||||
#include <memory>
|
|
||||||
|
|
||||||
#include "config.h"
|
|
||||||
namespace zkutil
|
|
||||||
{
|
|
||||||
class ZooKeeper;
|
|
||||||
using ZooKeeperPtr = std::shared_ptr<ZooKeeper>;
|
|
||||||
}
|
|
||||||
|
|
||||||
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<ContextSharedPart> shared_context);
|
|
||||||
SharedContextHolder(SharedContextHolder &&) noexcept;
|
|
||||||
|
|
||||||
SharedContextHolder & operator=(SharedContextHolder &&) noexcept;
|
|
||||||
|
|
||||||
ContextSharedPart * get() const { return shared.get(); }
|
|
||||||
void reset();
|
|
||||||
private:
|
|
||||||
std::unique_ptr<ContextSharedPart> 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<AsyncReadCounters> 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<Context>
|
|
||||||
{
|
|
||||||
private:
|
|
||||||
/// ContextData mutex
|
|
||||||
mutable SharedMutex mutex;
|
|
||||||
|
|
||||||
Context();
|
|
||||||
Context(const Context &);
|
|
||||||
|
|
||||||
std::unique_lock<SharedMutex> getGlobalLock() const;
|
|
||||||
|
|
||||||
std::shared_lock<SharedMutex> getGlobalSharedLock() const;
|
|
||||||
|
|
||||||
std::unique_lock<SharedMutex> getLocalLock() const;
|
|
||||||
|
|
||||||
std::shared_lock<SharedMutex> 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<Poco::Util::AbstractConfiguration>;
|
|
||||||
|
|
||||||
/// 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<Macros>::Version getMacros() const;
|
|
||||||
void setMacros(std::unique_ptr<Macros> && 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<FilesystemCacheLog> getFilesystemCacheLog() const;
|
|
||||||
std::shared_ptr<FilesystemReadPrefetchesLog> getFilesystemReadPrefetchesLog() const;
|
|
||||||
std::shared_ptr<BlobStorageLog> 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<AsyncReadCounters> 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<KeeperDispatcher> getKeeperDispatcher() const;
|
|
||||||
std::shared_ptr<KeeperDispatcher> 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;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -1,24 +0,0 @@
|
|||||||
#include <Core/Settings.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
IMPLEMENT_SETTINGS_TRAITS(SettingsTraits, LIST_OF_SETTINGS)
|
|
||||||
|
|
||||||
std::vector<String> Settings::getAllRegisteredNames() const
|
|
||||||
{
|
|
||||||
std::vector<String> 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);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
@ -1,19 +0,0 @@
|
|||||||
#include <Common/CurrentThread.h>
|
|
||||||
#include <Common/ThreadStatus.h>
|
|
||||||
|
|
||||||
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*/)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
@ -380,15 +380,6 @@ void SettingFieldString::readBinary(ReadBuffer & in)
|
|||||||
*this = std::move(str);
|
*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)) {}
|
SettingFieldMap::SettingFieldMap(const Field & f) : value(fieldToMap(f)) {}
|
||||||
|
|
||||||
String SettingFieldMap::toString() const
|
String SettingFieldMap::toString() const
|
||||||
@ -428,42 +419,6 @@ void SettingFieldMap::readBinary(ReadBuffer & in)
|
|||||||
*this = map;
|
*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
|
namespace
|
||||||
{
|
{
|
||||||
char stringToChar(const String & str)
|
char stringToChar(const String & str)
|
||||||
|
@ -247,12 +247,6 @@ struct SettingFieldString
|
|||||||
void readBinary(ReadBuffer & in);
|
void readBinary(ReadBuffer & in);
|
||||||
};
|
};
|
||||||
|
|
||||||
#ifdef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
#define NORETURN [[noreturn]]
|
|
||||||
#else
|
|
||||||
#define NORETURN
|
|
||||||
#endif
|
|
||||||
|
|
||||||
struct SettingFieldMap
|
struct SettingFieldMap
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
@ -269,11 +263,11 @@ public:
|
|||||||
operator const Map &() const { return value; } /// NOLINT
|
operator const Map &() const { return value; } /// NOLINT
|
||||||
explicit operator Field() const { return value; }
|
explicit operator Field() const { return value; }
|
||||||
|
|
||||||
NORETURN String toString() const;
|
String toString() const;
|
||||||
NORETURN void parseFromString(const String & str);
|
void parseFromString(const String & str);
|
||||||
|
|
||||||
NORETURN void writeBinary(WriteBuffer & out) const;
|
void writeBinary(WriteBuffer & out) const;
|
||||||
NORETURN void readBinary(ReadBuffer & in);
|
void readBinary(ReadBuffer & in);
|
||||||
};
|
};
|
||||||
|
|
||||||
#undef NORETURN
|
#undef NORETURN
|
||||||
|
@ -502,9 +502,7 @@ private:
|
|||||||
if (collectCrashLog)
|
if (collectCrashLog)
|
||||||
collectCrashLog(sig, thread_num, query_id, stack_trace);
|
collectCrashLog(sig, thread_num, query_id, stack_trace);
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
Context::getGlobalContextInstance()->handleCrash();
|
Context::getGlobalContextInstance()->handleCrash();
|
||||||
#endif
|
|
||||||
|
|
||||||
/// Send crash report to developers (if configured)
|
/// Send crash report to developers (if configured)
|
||||||
if (sig != SanitizerTrap)
|
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.
|
/// List changed settings.
|
||||||
if (!query_id.empty())
|
if (!query_id.empty())
|
||||||
{
|
{
|
||||||
@ -549,7 +545,6 @@ private:
|
|||||||
LOG_FATAL(log, "Changed settings: {}", changed_settings);
|
LOG_FATAL(log, "Changed settings: {}", changed_settings);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
|
|
||||||
/// When everything is done, we will try to send these error messages to the client.
|
/// When everything is done, we will try to send these error messages to the client.
|
||||||
if (thread_ptr)
|
if (thread_ptr)
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
#include "config.h"
|
#include "config.h"
|
||||||
#include <Common/config_version.h>
|
#include <Common/config_version.h>
|
||||||
|
|
||||||
#if USE_SENTRY && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_SENTRY
|
||||||
|
|
||||||
# include <sentry.h>
|
# include <sentry.h>
|
||||||
# include <cstdio>
|
# include <cstdio>
|
||||||
|
@ -78,7 +78,6 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
|
|||||||
|
|
||||||
std::unique_ptr<ReadBufferFromFileBase> buf;
|
std::unique_ptr<ReadBufferFromFileBase> buf;
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
if (with_file_cache)
|
if (with_file_cache)
|
||||||
{
|
{
|
||||||
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
|
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
|
||||||
@ -96,7 +95,6 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
|
|||||||
/* read_until_position */std::nullopt,
|
/* read_until_position */std::nullopt,
|
||||||
cache_log);
|
cache_log);
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
|
|
||||||
/// Can't wrap CachedOnDiskReadBufferFromFile in CachedInMemoryReadBufferFromFile because the
|
/// Can't wrap CachedOnDiskReadBufferFromFile in CachedInMemoryReadBufferFromFile because the
|
||||||
/// former doesn't support seeks.
|
/// former doesn't support seeks.
|
||||||
|
@ -195,7 +195,6 @@ public:
|
|||||||
/// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage)))
|
/// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage)))
|
||||||
String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); }
|
String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); }
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
/// Add a cache layer.
|
/// Add a cache layer.
|
||||||
/// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
|
/// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
|
||||||
/// There can be any number of cache layers:
|
/// 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.
|
/// Get names of all cache layers. Name is how cache is defined in configuration file.
|
||||||
NameSet getCacheLayersNames() const override;
|
NameSet getCacheLayersNames() const override;
|
||||||
#endif
|
|
||||||
|
|
||||||
bool supportsStat() const override { return metadata_storage->supportsStat(); }
|
bool supportsStat() const override { return metadata_storage->supportsStat(); }
|
||||||
struct stat stat(const String & path) const override;
|
struct stat stat(const String & path) const override;
|
||||||
|
@ -222,11 +222,7 @@ ObjectKeyWithMetadata DiskObjectStorageMetadata::popLastObject()
|
|||||||
|
|
||||||
bool DiskObjectStorageMetadata::getWriteFullObjectKeySetting()
|
bool DiskObjectStorageMetadata::getWriteFullObjectKeySetting()
|
||||||
{
|
{
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
return Context::getGlobalContextInstance()->getServerSettings().storage_metadata_write_full_object_key;
|
return Context::getGlobalContextInstance()->getServerSettings().storage_metadata_write_full_object_key;
|
||||||
#else
|
|
||||||
return false;
|
|
||||||
#endif
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,9 +2,7 @@
|
|||||||
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
|
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
|
||||||
#include <Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h>
|
#include <Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h>
|
||||||
#include <Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h>
|
#include <Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h>
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
#include <Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h>
|
#include <Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h>
|
||||||
#endif
|
|
||||||
#include <Disks/DiskLocal.h>
|
#include <Disks/DiskLocal.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
|
|
||||||
@ -135,7 +133,6 @@ void registerPlainRewritableMetadataStorage(MetadataStorageFactory & factory)
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
void registerMetadataStorageFromStaticFilesWebServer(MetadataStorageFactory & factory)
|
void registerMetadataStorageFromStaticFilesWebServer(MetadataStorageFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerMetadataStorageType("web", [](
|
factory.registerMetadataStorageType("web", [](
|
||||||
@ -147,7 +144,6 @@ void registerMetadataStorageFromStaticFilesWebServer(MetadataStorageFactory & fa
|
|||||||
return std::make_shared<MetadataStorageFromStaticFilesWebServer>(assert_cast<const WebObjectStorage &>(*object_storage));
|
return std::make_shared<MetadataStorageFromStaticFilesWebServer>(assert_cast<const WebObjectStorage &>(*object_storage));
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
|
|
||||||
void registerMetadataStorages()
|
void registerMetadataStorages()
|
||||||
{
|
{
|
||||||
@ -155,9 +151,7 @@ void registerMetadataStorages()
|
|||||||
registerMetadataStorageFromDisk(factory);
|
registerMetadataStorageFromDisk(factory);
|
||||||
registerPlainMetadataStorage(factory);
|
registerPlainMetadataStorage(factory);
|
||||||
registerPlainRewritableMetadataStorage(factory);
|
registerPlainRewritableMetadataStorage(factory);
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
registerMetadataStorageFromStaticFilesWebServer(factory);
|
registerMetadataStorageFromStaticFilesWebServer(factory);
|
||||||
#endif
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -7,19 +7,17 @@
|
|||||||
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||||
#endif
|
#endif
|
||||||
#if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_HDFS
|
||||||
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
|
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
|
||||||
#include <Storages/ObjectStorage/HDFS/HDFSCommon.h>
|
#include <Storages/ObjectStorage/HDFS/HDFSCommon.h>
|
||||||
#endif
|
#endif
|
||||||
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_AZURE_BLOB_STORAGE
|
||||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||||
#endif
|
#endif
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
#include <Disks/ObjectStorages/Web/WebObjectStorage.h>
|
#include <Disks/ObjectStorages/Web/WebObjectStorage.h>
|
||||||
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
|
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
|
||||||
#include <Disks/loadLocalDiskConfig.h>
|
#include <Disks/loadLocalDiskConfig.h>
|
||||||
#endif
|
|
||||||
#include <Disks/ObjectStorages/MetadataStorageFactory.h>
|
#include <Disks/ObjectStorages/MetadataStorageFactory.h>
|
||||||
#include <Disks/ObjectStorages/PlainObjectStorage.h>
|
#include <Disks/ObjectStorages/PlainObjectStorage.h>
|
||||||
#include <Disks/ObjectStorages/PlainRewritableObjectStorage.h>
|
#include <Disks/ObjectStorages/PlainRewritableObjectStorage.h>
|
||||||
@ -284,7 +282,7 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
|
|||||||
|
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_HDFS
|
||||||
void registerHDFSObjectStorage(ObjectStorageFactory & factory)
|
void registerHDFSObjectStorage(ObjectStorageFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerObjectStorageType(
|
factory.registerObjectStorageType(
|
||||||
@ -309,7 +307,7 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory)
|
|||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_AZURE_BLOB_STORAGE
|
||||||
void registerAzureObjectStorage(ObjectStorageFactory & factory)
|
void registerAzureObjectStorage(ObjectStorageFactory & factory)
|
||||||
{
|
{
|
||||||
auto creator = [](
|
auto creator = [](
|
||||||
@ -333,7 +331,6 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory)
|
|||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
void registerWebObjectStorage(ObjectStorageFactory & factory)
|
void registerWebObjectStorage(ObjectStorageFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerObjectStorageType("web", [](
|
factory.registerObjectStorageType("web", [](
|
||||||
@ -381,7 +378,6 @@ void registerLocalObjectStorage(ObjectStorageFactory & factory)
|
|||||||
factory.registerObjectStorageType("local_blob_storage", creator);
|
factory.registerObjectStorageType("local_blob_storage", creator);
|
||||||
factory.registerObjectStorageType("local", creator);
|
factory.registerObjectStorageType("local", creator);
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
|
|
||||||
void registerObjectStorages()
|
void registerObjectStorages()
|
||||||
{
|
{
|
||||||
@ -393,18 +389,16 @@ void registerObjectStorages()
|
|||||||
registerS3PlainRewritableObjectStorage(factory);
|
registerS3PlainRewritableObjectStorage(factory);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_HDFS
|
||||||
registerHDFSObjectStorage(factory);
|
registerHDFSObjectStorage(factory);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_AZURE_BLOB_STORAGE
|
||||||
registerAzureObjectStorage(factory);
|
registerAzureObjectStorage(factory);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
registerWebObjectStorage(factory);
|
registerWebObjectStorage(factory);
|
||||||
registerLocalObjectStorage(factory);
|
registerLocalObjectStorage(factory);
|
||||||
#endif
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,14 +1,14 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
|
#include "config.h"
|
||||||
|
|
||||||
#if USE_AWS_S3
|
#if USE_AWS_S3
|
||||||
# include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
# include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||||
#endif
|
#endif
|
||||||
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_AZURE_BLOB_STORAGE
|
||||||
# include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
# include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||||
#endif
|
#endif
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
|
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
|
||||||
#endif
|
|
||||||
#include <Disks/ObjectStorages/MetadataStorageMetrics.h>
|
#include <Disks/ObjectStorages/MetadataStorageMetrics.h>
|
||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
@ -42,7 +42,7 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create<S3ObjectStorage, Me
|
|||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_AZURE_BLOB_STORAGE
|
||||||
template <>
|
template <>
|
||||||
inline MetadataStorageMetrics MetadataStorageMetrics::create<AzureObjectStorage, MetadataStorageType::PlainRewritable>()
|
inline MetadataStorageMetrics MetadataStorageMetrics::create<AzureObjectStorage, MetadataStorageType::PlainRewritable>()
|
||||||
{
|
{
|
||||||
@ -53,7 +53,6 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create<AzureObjectStorage,
|
|||||||
}
|
}
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
template <>
|
template <>
|
||||||
inline MetadataStorageMetrics MetadataStorageMetrics::create<LocalObjectStorage, MetadataStorageType::PlainRewritable>()
|
inline MetadataStorageMetrics MetadataStorageMetrics::create<LocalObjectStorage, MetadataStorageType::PlainRewritable>()
|
||||||
{
|
{
|
||||||
@ -62,6 +61,5 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create<LocalObjectStorage,
|
|||||||
.directory_removed = ProfileEvents::DiskPlainRewritableLocalDirectoryRemoved,
|
.directory_removed = ProfileEvents::DiskPlainRewritableLocalDirectoryRemoved,
|
||||||
.directory_map_size = CurrentMetrics::DiskPlainRewritableLocalDirectoryMapSize};
|
.directory_map_size = CurrentMetrics::DiskPlainRewritableLocalDirectoryMapSize};
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -17,8 +17,6 @@ void registerDiskCache(DiskFactory & factory, bool global_skip_access_check);
|
|||||||
void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_check);
|
void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_check);
|
||||||
|
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
|
|
||||||
void registerDisks(bool global_skip_access_check)
|
void registerDisks(bool global_skip_access_check)
|
||||||
{
|
{
|
||||||
auto & factory = DiskFactory::instance();
|
auto & factory = DiskFactory::instance();
|
||||||
@ -34,17 +32,4 @@ void registerDisks(bool global_skip_access_check)
|
|||||||
registerDiskObjectStorage(factory, global_skip_access_check);
|
registerDiskObjectStorage(factory, global_skip_access_check);
|
||||||
}
|
}
|
||||||
|
|
||||||
#else
|
|
||||||
|
|
||||||
void registerDisks(bool global_skip_access_check)
|
|
||||||
{
|
|
||||||
auto & factory = DiskFactory::instance();
|
|
||||||
|
|
||||||
registerDiskLocal(factory, global_skip_access_check);
|
|
||||||
|
|
||||||
registerDiskObjectStorage(factory, global_skip_access_check);
|
|
||||||
}
|
|
||||||
|
|
||||||
#endif
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -56,7 +56,6 @@ void BlobStorageLogWriter::addEvent(
|
|||||||
|
|
||||||
BlobStorageLogWriterPtr BlobStorageLogWriter::create(const String & disk_name)
|
BlobStorageLogWriterPtr BlobStorageLogWriter::create(const String & disk_name)
|
||||||
{
|
{
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD /// Keeper standalone build doesn't have a context
|
|
||||||
if (auto blob_storage_log = Context::getGlobalContextInstance()->getBlobStorageLog())
|
if (auto blob_storage_log = Context::getGlobalContextInstance()->getBlobStorageLog())
|
||||||
{
|
{
|
||||||
auto log_writer = std::make_shared<BlobStorageLogWriter>(std::move(blob_storage_log));
|
auto log_writer = std::make_shared<BlobStorageLogWriter>(std::move(blob_storage_log));
|
||||||
@ -67,7 +66,6 @@ BlobStorageLogWriterPtr BlobStorageLogWriter::create(const String & disk_name)
|
|||||||
|
|
||||||
return log_writer;
|
return log_writer;
|
||||||
}
|
}
|
||||||
#endif
|
|
||||||
return {};
|
return {};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1,7 +1,5 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
|
|
||||||
|
|
||||||
#include <base/types.h>
|
#include <base/types.h>
|
||||||
#include <Common/isLocalAddress.h>
|
#include <Common/isLocalAddress.h>
|
||||||
#include <Common/MultiVersion.h>
|
#include <Common/MultiVersion.h>
|
||||||
@ -1451,9 +1449,3 @@ struct HTTPContext : public IHTTPContext
|
|||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
#else
|
|
||||||
|
|
||||||
#include <Coordination/Standalone/Context.h>
|
|
||||||
|
|
||||||
#endif
|
|
||||||
|
@ -18,9 +18,6 @@ void PrometheusRequestHandler::handleRequest(HTTPServerRequest & request, HTTPSe
|
|||||||
{
|
{
|
||||||
try
|
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();
|
const auto & config = server.config();
|
||||||
unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
|
unsigned keep_alive_timeout = config.getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT);
|
||||||
|
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
#include <Server/ProtocolServerAdapter.h>
|
#include <Server/ProtocolServerAdapter.h>
|
||||||
#include <Server/TCPServer.h>
|
#include <Server/TCPServer.h>
|
||||||
|
|
||||||
#if USE_GRPC && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_GRPC
|
||||||
#include <Server/GRPCServer.h>
|
#include <Server/GRPCServer.h>
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
@ -37,7 +37,7 @@ ProtocolServerAdapter::ProtocolServerAdapter(
|
|||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
#if USE_GRPC && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD)
|
#if USE_GRPC
|
||||||
class ProtocolServerAdapter::GRPCServerAdapterImpl : public Impl
|
class ProtocolServerAdapter::GRPCServerAdapterImpl : public Impl
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
|
@ -23,7 +23,7 @@ public:
|
|||||||
ProtocolServerAdapter & operator =(ProtocolServerAdapter && src) = default;
|
ProtocolServerAdapter & operator =(ProtocolServerAdapter && src) = default;
|
||||||
ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr<TCPServer> tcp_server_);
|
ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr<TCPServer> 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<GRPCServer> grpc_server_);
|
ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr<GRPCServer> grpc_server_);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user