Merge branch 'master' into fix_ec_issue

This commit is contained in:
李扬 2023-07-03 20:41:04 +08:00 committed by GitHub
commit a8cf9318fd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
100 changed files with 2983 additions and 967 deletions

View File

@ -121,8 +121,6 @@ jobs:
docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||:
sudo rm -fr "$TEMP_PATH" "$CACHES_PATH"
SonarCloud:
# TODO: Remove if: whenever SonarCloud supports c++23
if: ${{ false }}
runs-on: [self-hosted, builder]
env:
SONAR_SCANNER_VERSION: 4.8.0.2856
@ -159,7 +157,7 @@ jobs:
- name: Set Up Build Tools
run: |
sudo apt-get update
sudo apt-get install -yq git cmake ccache ninja-build python3 yasm
sudo apt-get install -yq git cmake ccache ninja-build python3 yasm nasm
sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)"
- name: Run build-wrapper
run: |
@ -178,4 +176,5 @@ jobs:
--define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \
--define sonar.projectKey="ClickHouse_ClickHouse" \
--define sonar.organization="clickhouse-java" \
--define sonar.exclusions="**/*.java,**/*.ts,**/*.js,**/*.css,**/*.sql" \
--define sonar.cfamily.cpp23.enabled=true \
--define sonar.exclusions="**/*.java,**/*.ts,**/*.js,**/*.css,**/*.sql"

View File

@ -23,11 +23,11 @@ curl https://clickhouse.com/ | sh
## Upcoming Events
* [**v23.6 Release Webinar**](https://clickhouse.com/company/events/v23-6-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-06) - Jun 29 - 23.6 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release.
* [**ClickHouse Meetup in Paris**](https://www.meetup.com/clickhouse-france-user-group/events/294283460) - Jul 4
* [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18
* [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19
* [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20
* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27
* [**ClickHouse Meetup in Paris**](https://www.meetup.com/clickhouse-france-user-group/events/294283460) - Sep 12
Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com.

View File

@ -120,11 +120,12 @@
"docker/test/base": {
"name": "clickhouse/test-base",
"dependent": [
"docker/test/stateless",
"docker/test/integration/base",
"docker/test/fuzzer",
"docker/test/integration/base",
"docker/test/keeper-jepsen",
"docker/test/server-jepsen"
"docker/test/server-jepsen",
"docker/test/sqllogic",
"docker/test/stateless"
]
},
"docker/test/integration/kerberized_hadoop": {

View File

@ -13,6 +13,7 @@ RUN apt-get update --yes \
sqlite3 \
unixodbc \
unixodbc-dev \
odbcinst \
sudo \
&& apt-get clean

View File

@ -189,6 +189,7 @@ rg -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \
-e "Authentication failed" \
-e "Cannot flush" \
-e "Container already exists" \
-e "doesn't have metadata version on disk" \
clickhouse-server.upgrade.log \
| grep -av -e "_repl_01111_.*Mapping for table with UUID" \
| zgrep -Fa "<Error>" > /test_output/upgrade_error_messages.txt \

View File

@ -9,6 +9,7 @@
#include <Poco/AutoPtr.h>
#include <Poco/Logger.h>
#include <Common/logger_useful.h>
#include <Disks/DiskLocal.h>
int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
@ -39,8 +40,9 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
try
{
auto keeper_context = std::make_shared<KeeperContext>();
keeper_context->digest_enabled = true;
auto keeper_context = std::make_shared<KeeperContext>(true);
keeper_context->setDigestEnabled(true);
keeper_context->setSnapshotDisk(std::make_shared<DiskLocal>("Keeper-snapshots", options["output-dir"].as<std::string>(), 0));
DB::KeeperStorage storage(/* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false);
@ -51,10 +53,10 @@ int mainEntryClickHouseKeeperConverter(int argc, char ** argv)
DB::SnapshotMetadataPtr snapshot_meta = std::make_shared<DB::SnapshotMetadata>(storage.getZXID(), 1, std::make_shared<nuraft::cluster_config>());
DB::KeeperStorageSnapshot snapshot(&storage, snapshot_meta);
DB::KeeperSnapshotManager manager(options["output-dir"].as<std::string>(), 1, keeper_context);
DB::KeeperSnapshotManager manager(1, keeper_context);
auto snp = manager.serializeSnapshotToBuffer(snapshot);
auto path = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID());
std::cout << "Snapshot serialized to path:" << path << std::endl;
auto file_info = manager.serializeSnapshotBufferToDisk(*snp, storage.getZXID());
std::cout << "Snapshot serialized to path:" << fs::path(file_info.disk->getPath()) / file_info.path << std::endl;
}
catch (...)
{

View File

@ -48,10 +48,10 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManager.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperSnapshotManagerS3.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateMachine.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperContext.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStateManager.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperAsynchronousMetrics.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/TinyContext.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/pathUtils.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SessionExpiryQueue.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/SummingStateMachine.cpp
@ -60,10 +60,14 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsFields.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BaseSettings.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/ServerSettings.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/Field.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/SettingsEnums.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/ServerUUID.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/UUID.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Core/BackgroundSchedulePool.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/IO/ReadBuffer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/KeeperTCPHandler.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Server/TCPServer.cpp
@ -95,6 +99,10 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/ICompressionCodec.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Compression/LZ4_decompress_faster.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/CurrentThread.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollections.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/NamedCollections/NamedCollectionConfiguration.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/IKeeper.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/TestKeeper.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperCommon.cpp
@ -105,11 +113,58 @@ if (BUILD_STANDALONE_KEEPER)
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperLock.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Common/ZooKeeper/ZooKeeperNodeCache.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/registerDisks.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IDisk.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskFactory.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskSelector.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskLocal.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/DiskLocalCheckThread.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/LocalDirectorySyncGuard.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/TemporaryFileOnDisk.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/loadLocalDiskConfig.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/IObjectStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataFromDiskTransactionState.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/DiskObjectStorageCommon.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/ObjectStorageIterator.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/StoredObject.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/registerDiskS3.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/S3Capabilities.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/diskSettings.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/ProxyListConfiguration.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/ObjectStorages/S3/ProxyResolverConfiguration.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/createReadBufferFromFileBase.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ReadBufferFromRemoteFSGather.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/IOUringReader.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferFromTemporaryFile.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/WriteBufferWithFinalizeCallback.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/AsynchronousBoundedReadBuffer.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/getThreadPoolReader.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolRemoteFSReader.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Disks/IO/ThreadPoolReader.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Storages/StorageS3Settings.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/BaseDaemon.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/SentryWriter.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Daemon/GraphiteWriter.cpp
${CMAKE_CURRENT_BINARY_DIR}/../../src/Daemon/GitHash.generated.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/Context.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/Settings.cpp
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Standalone/ThreadStatusExt.cpp
Keeper.cpp
clickhouse-keeper.cpp
)
@ -132,10 +187,6 @@ if (BUILD_STANDALONE_KEEPER)
target_compile_definitions (clickhouse-keeper PRIVATE -DCLICKHOUSE_PROGRAM_STANDALONE_BUILD)
target_compile_definitions (clickhouse-keeper PUBLIC -DWITHOUT_TEXT_LOG)
target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/../../src") # uses includes from src directory
target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src/Core/include") # uses some includes from core
target_include_directories(clickhouse-keeper PUBLIC "${CMAKE_CURRENT_BINARY_DIR}/../../src") # uses some includes from common
if (ENABLE_CLICKHOUSE_KEEPER_CLIENT AND TARGET ch_rust::skim)
target_link_libraries(clickhouse-keeper PRIVATE ch_rust::skim)
endif()

View File

@ -24,6 +24,8 @@
#include <sys/stat.h>
#include <pwd.h>
#include <Interpreters/Context.h>
#include <Coordination/FourLetterCommand.h>
#include <Coordination/KeeperAsynchronousMetrics.h>
@ -45,6 +47,8 @@
#include <Server/ProtocolServerAdapter.h>
#include <Server/KeeperTCPHandlerFactory.h>
#include <Disks/registerDisks.h>
int mainEntryClickHouseKeeper(int argc, char ** argv)
{
@ -201,9 +205,12 @@ void Keeper::defineOptions(Poco::Util::OptionSet & options)
BaseDaemon::defineOptions(options);
}
struct Keeper::KeeperHTTPContext : public IHTTPContext
namespace
{
explicit KeeperHTTPContext(TinyContextPtr context_)
struct KeeperHTTPContext : public IHTTPContext
{
explicit KeeperHTTPContext(ContextPtr context_)
: context(std::move(context_))
{}
@ -247,12 +254,14 @@ struct Keeper::KeeperHTTPContext : public IHTTPContext
return {context->getConfigRef().getInt64("keeper_server.http_send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0};
}
TinyContextPtr context;
ContextPtr context;
};
HTTPContextPtr Keeper::httpContext()
HTTPContextPtr httpContext()
{
return std::make_shared<KeeperHTTPContext>(tiny_context);
return std::make_shared<KeeperHTTPContext>(Context::getGlobalContextInstance());
}
}
int Keeper::main(const std::vector<std::string> & /*args*/)
@ -316,10 +325,21 @@ try
std::mutex servers_lock;
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
tiny_context = std::make_shared<TinyContext>();
auto shared_context = Context::createShared();
auto global_context = Context::createGlobal(shared_context.get());
global_context->makeGlobalContext();
global_context->setPath(path);
global_context->setRemoteHostFilter(config());
if (config().has("macros"))
global_context->setMacros(std::make_unique<Macros>(config(), "macros", log));
registerDisks(/*global_skip_access_check=*/false);
/// This object will periodically calculate some metrics.
KeeperAsynchronousMetrics async_metrics(
tiny_context,
global_context,
config().getUInt("asynchronous_metrics_update_period_s", 1),
[&]() -> std::vector<ProtocolServerMetrics>
{
@ -344,12 +364,12 @@ try
}
/// Initialize keeper RAFT. Do nothing if no keeper_server in config.
tiny_context->initializeKeeperDispatcher(/* start_async = */ true);
FourLetterCommandFactory::registerCommands(*tiny_context->getKeeperDispatcher());
global_context->initializeKeeperDispatcher(/* start_async = */ true);
FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher());
auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration &
auto config_getter = [&] () -> const Poco::Util::AbstractConfiguration &
{
return tiny_context->getConfigRef();
return global_context->getConfigRef();
};
auto tcp_receive_timeout = config().getInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC);
@ -371,7 +391,7 @@ try
"Keeper (tcp): " + address.toString(),
std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory(
config_getter, tiny_context->getKeeperDispatcher(),
config_getter, global_context->getKeeperDispatcher(),
tcp_receive_timeout, tcp_send_timeout, false), server_pool, socket));
});
@ -389,7 +409,7 @@ try
"Keeper with secure protocol (tcp_secure): " + address.toString(),
std::make_unique<TCPServer>(
new KeeperTCPHandlerFactory(
config_getter, tiny_context->getKeeperDispatcher(),
config_getter, global_context->getKeeperDispatcher(),
tcp_receive_timeout, tcp_send_timeout, true), server_pool, socket));
#else
UNUSED(port);
@ -441,7 +461,7 @@ try
[&](ConfigurationPtr config, bool /* initial_loading */)
{
if (config->has("keeper_server"))
tiny_context->updateKeeperConfiguration(*config);
global_context->updateKeeperConfiguration(*config);
},
/* already_loaded = */ false); /// Reload it right now (initial loading)
@ -472,7 +492,7 @@ try
else
LOG_INFO(log, "Closed connections to Keeper.");
tiny_context->shutdownKeeperDispatcher();
global_context->shutdownKeeperDispatcher();
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
server_pool.joinAll();

View File

@ -1,9 +1,7 @@
#pragma once
#include <Server/IServer.h>
#include <Server/HTTP/HTTPContext.h>
#include <Daemon/BaseDaemon.h>
#include <Coordination/TinyContext.h>
namespace Poco
{
@ -68,11 +66,6 @@ protected:
std::string getDefaultConfigFileName() const override;
private:
TinyContextPtr tiny_context;
struct KeeperHTTPContext;
HTTPContextPtr httpContext();
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
using CreateServerFunc = std::function<void(UInt16)>;

View File

@ -1,18 +1,19 @@
#include <filesystem>
#include <Coordination/Changelog.h>
#include <Disks/DiskLocal.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <IO/ZstdDeflatingAppendableWriteBuffer.h>
#include <base/errnoToString.h>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
#include <Common/filesystemHelpers.h>
#include <Common/Exception.h>
#include <Common/SipHash.h>
#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>
#include <IO/WriteBufferFromFile.h>
#include <base/errnoToString.h>
#include <libnuraft/log_val_type.hxx>
@ -24,20 +25,41 @@ namespace ErrorCodes
extern const int CHECKSUM_DOESNT_MATCH;
extern const int CORRUPTED_DATA;
extern const int UNKNOWN_FORMAT_VERSION;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
namespace
{
constexpr std::string_view tmp_prefix = "tmp_";
void moveFileBetweenDisks(DiskPtr disk_from, ChangelogFileDescriptionPtr description, DiskPtr disk_to, const std::string & path_to)
{
/// we use empty file with prefix tmp_ to detect incomplete copies
/// if a copy is complete we don't care from which disk we use the same file
/// so it's okay if a failure happens after removing of tmp file but before we remove
/// the changelog from the source disk
auto from_path = fs::path(description->path);
auto tmp_changelog_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string());
{
auto buf = disk_to->writeFile(tmp_changelog_name);
buf->finalize();
}
disk_from->copyFile(from_path, *disk_to, path_to, {});
disk_to->removeFile(tmp_changelog_name);
disk_from->removeFile(description->path);
description->path = path_to;
description->disk = disk_to;
}
constexpr auto DEFAULT_PREFIX = "changelog";
std::string formatChangelogPath(
const std::string & prefix, const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension)
inline std::string
formatChangelogPath(const std::string & name_prefix, uint64_t from_index, uint64_t to_index, const std::string & extension)
{
std::filesystem::path path(prefix);
path /= std::filesystem::path(fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension));
return path;
return fmt::format("{}_{}_{}.{}", name_prefix, from_index, to_index, extension);
}
ChangelogFileDescriptionPtr getChangelogFileDescription(const std::filesystem::path & path)
@ -89,17 +111,19 @@ class ChangelogWriter
public:
ChangelogWriter(
std::map<uint64_t, ChangelogFileDescriptionPtr> & existing_changelogs_,
const std::filesystem::path & changelogs_dir_,
KeeperContextPtr keeper_context_,
LogFileSettings log_file_settings_)
: existing_changelogs(existing_changelogs_)
, log_file_settings(log_file_settings_)
, changelogs_dir(changelogs_dir_)
, keeper_context(std::move(keeper_context_))
, log(&Poco::Logger::get("Changelog"))
{
}
void setFile(ChangelogFileDescriptionPtr file_description, WriteMode mode)
{
auto disk = getDisk();
try
{
if (mode == WriteMode::Append && file_description->expectedEntriesCountInLog() != log_file_settings.rotate_interval)
@ -110,7 +134,7 @@ public:
file_description->expectedEntriesCountInLog());
// we have a file we need to finalize first
if (tryGetFileBuffer() && prealloc_done)
if (tryGetFileBaseBuffer() && prealloc_done)
{
finalizeCurrentFile();
@ -118,27 +142,55 @@ public:
// if we wrote at least 1 log in the log file we can rename the file to reflect correctly the
// contained logs
// file can be deleted from disk earlier by compaction
if (!current_file_description->deleted && last_index_written
&& *last_index_written != current_file_description->to_log_index)
if (!current_file_description->deleted)
{
auto new_path = formatChangelogPath(
changelogs_dir,
current_file_description->prefix,
current_file_description->from_log_index,
*last_index_written,
current_file_description->extension);
std::filesystem::rename(current_file_description->path, new_path);
current_file_description->path = std::move(new_path);
auto log_disk = current_file_description->disk;
const auto & path = current_file_description->path;
std::string new_path = path;
if (last_index_written && *last_index_written != current_file_description->to_log_index)
{
new_path = formatChangelogPath(
current_file_description->prefix,
current_file_description->from_log_index,
*last_index_written,
current_file_description->extension);
}
if (disk == log_disk)
{
if (path != new_path)
{
try
{
disk->moveFile(path, new_path);
}
catch (...)
{
tryLogCurrentException(log, fmt::format("File rename failed on disk {}", disk->getName()));
}
current_file_description->path = std::move(new_path);
}
}
else
{
moveFileBetweenDisks(log_disk, current_file_description, disk, new_path);
}
}
}
file_buf = std::make_unique<WriteBufferFromFile>(
file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode == WriteMode::Rewrite ? -1 : (O_APPEND | O_CREAT | O_WRONLY));
auto latest_log_disk = getLatestLogDisk();
assert(file_description->disk == latest_log_disk);
file_buf = latest_log_disk->writeFile(file_description->path, DBMS_DEFAULT_BUFFER_SIZE, mode);
assert(file_buf);
last_index_written.reset();
current_file_description = std::move(file_description);
if (log_file_settings.compress_logs)
compressed_buffer = std::make_unique<ZstdDeflatingAppendableWriteBuffer>(std::move(file_buf), /* compression level = */ 3, /* append_to_existing_file_ = */ mode == WriteMode::Append);
compressed_buffer = std::make_unique<ZstdDeflatingAppendableWriteBuffer>(
std::move(file_buf),
/* compressi)on level = */ 3,
/* append_to_existing_file_ = */ mode == WriteMode::Append,
[latest_log_disk, path = current_file_description->path] { return latest_log_disk->readFile(path); });
prealloc_done = false;
}
@ -149,12 +201,12 @@ public:
}
}
bool isFileSet() const { return tryGetFileBuffer() != nullptr; }
/// There is bug when compressed_buffer has value, file_buf's ownership transfer to compressed_buffer
bool isFileSet() const { return compressed_buffer != nullptr || file_buf != nullptr; }
bool appendRecord(ChangelogRecord && record)
{
const auto * file_buffer = tryGetFileBuffer();
const auto * file_buffer = tryGetFileBaseBuffer();
assert(file_buffer && current_file_description);
assert(record.header.index - getStartIndex() <= current_file_description->expectedEntriesCountInLog());
@ -211,7 +263,7 @@ public:
void flush()
{
auto * file_buffer = tryGetFileBuffer();
auto * file_buffer = tryGetFileBaseBuffer();
if (file_buffer)
{
/// Fsync file system if needed
@ -236,12 +288,12 @@ public:
new_description->from_log_index = new_start_log_index;
new_description->to_log_index = new_start_log_index + log_file_settings.rotate_interval - 1;
new_description->extension = "bin";
new_description->disk = getLatestLogDisk();
if (log_file_settings.compress_logs)
new_description->extension += "." + toContentEncodingName(CompressionMethod::Zstd);
new_description->path = formatChangelogPath(
changelogs_dir,
new_description->prefix,
new_start_log_index,
new_start_log_index + log_file_settings.rotate_interval - 1,
@ -260,17 +312,15 @@ public:
}
private:
void finalizeCurrentFile()
{
const auto * file_buffer = tryGetFileBuffer();
assert(file_buffer && prealloc_done);
assert(prealloc_done);
assert(current_file_description);
// compact can delete the file and we don't need to do anything
if (current_file_description->deleted)
{
LOG_WARNING(log, "Log {} is already deleted", file_buffer->getFileName());
LOG_WARNING(log, "Log {} is already deleted", current_file_description->path);
return;
}
@ -279,27 +329,36 @@ private:
flush();
if (log_file_settings.max_size != 0)
const auto * file_buffer = tryGetFileBuffer();
if (log_file_settings.max_size != 0 && file_buffer)
{
int res = -1;
do
{
res = ftruncate(file_buffer->getFD(), initial_file_size + file_buffer->count());
}
while (res < 0 && errno == EINTR);
} while (res < 0 && errno == EINTR);
if (res != 0)
LOG_WARNING(log, "Could not ftruncate file. Error: {}, errno: {}", errnoToString(), errno);
}
if (log_file_settings.compress_logs)
{
compressed_buffer.reset();
}
else
{
chassert(file_buf);
file_buf->finalize();
file_buf.reset();
}
}
WriteBuffer & getBuffer()
{
/// TODO: unify compressed_buffer and file_buf,
/// compressed_buffer can use its NestedBuffer directly if compress_logs=false
if (compressed_buffer)
return *compressed_buffer;
@ -319,38 +378,42 @@ private:
return *file_buffer;
}
const WriteBufferFromFile * tryGetFileBuffer() const
{
return const_cast<ChangelogWriter *>(this)->tryGetFileBuffer();
}
const WriteBufferFromFile * tryGetFileBuffer() const { return const_cast<ChangelogWriter *>(this)->tryGetFileBuffer(); }
WriteBufferFromFile * tryGetFileBuffer()
{
if (compressed_buffer)
return dynamic_cast<WriteBufferFromFile *>(compressed_buffer->getNestedBuffer());
if (file_buf)
return file_buf.get();
return dynamic_cast<WriteBufferFromFile *>(file_buf.get());
}
return nullptr;
WriteBufferFromFileBase * tryGetFileBaseBuffer()
{
if (compressed_buffer)
return dynamic_cast<WriteBufferFromFileBase *>(compressed_buffer->getNestedBuffer());
return file_buf.get();
}
void tryPreallocateForFile()
{
if (log_file_settings.max_size == 0)
const auto * file_buffer = tryGetFileBuffer();
if (log_file_settings.max_size == 0 || !file_buffer)
{
initial_file_size = 0;
prealloc_done = true;
return;
}
const auto & file_buffer = getFileBuffer();
#ifdef OS_LINUX
{
int res = -1;
do
{
res = fallocate(file_buffer.getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size);
res = fallocate(
file_buffer->getFD(), FALLOC_FL_KEEP_SIZE, 0, log_file_settings.max_size + log_file_settings.overallocate_size);
} while (res < 0 && errno == EINTR);
if (res != 0)
@ -365,15 +428,21 @@ private:
}
}
#endif
initial_file_size = getSizeFromFileDescriptor(file_buffer.getFD());
initial_file_size = getSizeFromFileDescriptor(file_buffer->getFD());
prealloc_done = true;
}
DiskPtr getLatestLogDisk() const { return keeper_context->getLatestLogDisk(); }
DiskPtr getDisk() const { return keeper_context->getLogDisk(); }
bool isLocalDisk() const { return dynamic_cast<DiskLocal *>(getDisk().get()) != nullptr; }
std::map<uint64_t, ChangelogFileDescriptionPtr> & existing_changelogs;
ChangelogFileDescriptionPtr current_file_description{nullptr};
std::unique_ptr<WriteBufferFromFile> file_buf;
std::unique_ptr<WriteBufferFromFileBase> file_buf;
std::optional<uint64_t> last_index_written;
size_t initial_file_size{0};
@ -383,7 +452,7 @@ private:
LogFileSettings log_file_settings;
const std::filesystem::path changelogs_dir;
KeeperContextPtr keeper_context;
Poco::Logger * const log;
};
@ -413,10 +482,10 @@ struct ChangelogReadResult
class ChangelogReader
{
public:
explicit ChangelogReader(const std::string & filepath_) : filepath(filepath_)
explicit ChangelogReader(DiskPtr disk_, const std::string & filepath_) : disk(disk_), filepath(filepath_)
{
auto compression_method = chooseCompressionMethod(filepath, "");
auto read_buffer_from_file = std::make_unique<ReadBufferFromFile>(filepath);
auto read_buffer_from_file = disk->readFile(filepath);
read_buf = wrapReadBufferWithCompressionMethod(std::move(read_buffer_from_file), compression_method);
}
@ -512,37 +581,103 @@ public:
}
private:
DiskPtr disk;
std::string filepath;
std::unique_ptr<ReadBuffer> read_buf;
};
Changelog::Changelog(
const std::string & changelogs_dir_,
Poco::Logger * log_,
LogFileSettings log_file_settings)
: changelogs_dir(changelogs_dir_)
, changelogs_detached_dir(changelogs_dir / "detached")
Changelog::Changelog(Poco::Logger * log_, LogFileSettings log_file_settings, KeeperContextPtr keeper_context_)
: changelogs_detached_dir("detached")
, rotate_interval(log_file_settings.rotate_interval)
, log(log_)
, write_operations(std::numeric_limits<size_t>::max())
, append_completion_queue(std::numeric_limits<size_t>::max())
, keeper_context(std::move(keeper_context_))
{
/// Load all files in changelog directory
namespace fs = std::filesystem;
if (!fs::exists(changelogs_dir))
fs::create_directories(changelogs_dir);
for (const auto & p : fs::directory_iterator(changelogs_dir))
if (auto latest_log_disk = getLatestLogDisk();
log_file_settings.force_sync && dynamic_cast<const DiskLocal *>(latest_log_disk.get()) == nullptr)
{
if (p == changelogs_detached_dir)
continue;
auto file_description = getChangelogFileDescription(p.path());
existing_changelogs[file_description->from_log_index] = std::move(file_description);
throw DB::Exception(
DB::ErrorCodes::BAD_ARGUMENTS,
"force_sync is set to true for logs but disk '{}' cannot satisfy such guarantee because it's not of type DiskLocal.\n"
"If you want to use force_sync and same disk for all logs, please set keeper_server.log_storage_disk to a local disk.\n"
"If you want to use force_sync and different disk only for old logs, please set 'keeper_server.log_storage_disk' to any "
"supported disk and 'keeper_server.latest_log_storage_disk' to a local disk.\n"
"Otherwise, disable force_sync",
latest_log_disk->getName());
}
/// Load all files on changelog disks
const auto load_from_disk = [&](const auto & disk)
{
LOG_TRACE(log, "Reading from disk {}", disk->getName());
std::unordered_map<std::string, std::string> incomplete_files;
const auto clean_incomplete_file = [&](const auto & file_path)
{
if (auto incomplete_it = incomplete_files.find(fs::path(file_path).filename()); incomplete_it != incomplete_files.end())
{
LOG_TRACE(log, "Removing {} from {}", file_path, disk->getName());
disk->removeFile(file_path);
disk->removeFile(incomplete_it->second);
incomplete_files.erase(incomplete_it);
return true;
}
return false;
};
std::vector<std::string> changelog_files;
for (auto it = disk->iterateDirectory(""); it->isValid(); it->next())
{
if (it->name() == changelogs_detached_dir)
continue;
if (it->name().starts_with(tmp_prefix))
{
incomplete_files.emplace(it->name().substr(tmp_prefix.size()), it->path());
continue;
}
if (clean_incomplete_file(it->path()))
continue;
changelog_files.push_back(it->path());
}
for (const auto & changelog_file : changelog_files)
{
if (clean_incomplete_file(fs::path(changelog_file).filename()))
continue;
auto file_description = getChangelogFileDescription(changelog_file);
file_description->disk = disk;
LOG_TRACE(log, "Found {} on {}", changelog_file, disk->getName());
auto [changelog_it, inserted] = existing_changelogs.insert_or_assign(file_description->from_log_index, std::move(file_description));
if (!inserted)
LOG_WARNING(log, "Found duplicate entries for {}, will use the entry from {}", changelog_it->second->path, disk->getName());
}
for (const auto & [name, path] : incomplete_files)
disk->removeFile(path);
};
/// Load all files from old disks
for (const auto & disk : keeper_context->getOldLogDisks())
load_from_disk(disk);
auto disk = getDisk();
load_from_disk(disk);
auto latest_log_disk = getLatestLogDisk();
if (disk != latest_log_disk)
load_from_disk(latest_log_disk);
if (existing_changelogs.empty())
LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", changelogs_dir.generic_string());
LOG_WARNING(log, "No logs exists in {}. It's Ok if it's the first run of clickhouse-keeper.", disk->getPath());
clean_log_thread = ThreadFromGlobalPool([this] { cleanLogThread(); });
@ -550,8 +685,7 @@ Changelog::Changelog(
append_completion_thread = ThreadFromGlobalPool([this] { appendCompletionThread(); });
current_writer = std::make_unique<ChangelogWriter>(
existing_changelogs, changelogs_dir, log_file_settings);
current_writer = std::make_unique<ChangelogWriter>(existing_changelogs, keeper_context, log_file_settings);
}
void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uint64_t logs_to_keep)
@ -623,7 +757,7 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
break;
}
ChangelogReader reader(changelog_description.path);
ChangelogReader reader(changelog_description.disk, changelog_description.path);
last_log_read_result = reader.readChangelog(logs, start_to_read_from, log);
last_log_read_result->log_start_index = changelog_description.from_log_index;
@ -684,13 +818,13 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
assert(existing_changelogs.find(last_log_read_result->log_start_index) != existing_changelogs.end());
assert(existing_changelogs.find(last_log_read_result->log_start_index)->first == existing_changelogs.rbegin()->first);
/// Continue to write into incomplete existing log if it doesn't finished with error
/// Continue to write into incomplete existing log if it didn't finish with error
const auto & description = existing_changelogs[last_log_read_result->log_start_index];
if (last_log_read_result->last_read_index == 0 || last_log_read_result->error) /// If it's broken log then remove it
{
LOG_INFO(log, "Removing chagelog {} because it's empty or read finished with error", description->path);
std::filesystem::remove(description->path);
description->disk->removeFile(description->path);
existing_changelogs.erase(last_log_read_result->log_start_index);
std::erase_if(logs, [last_log_read_result](const auto & item) { return item.first >= last_log_read_result->log_start_index; });
}
@ -699,55 +833,124 @@ void Changelog::readChangelogAndInitWriter(uint64_t last_commited_log_index, uin
initWriter(description);
}
}
else if (last_log_read_result.has_value())
{
/// check if we need to move completed log to another disk
auto latest_log_disk = getLatestLogDisk();
auto disk = getDisk();
auto & description = existing_changelogs.at(last_log_read_result->log_start_index);
if (latest_log_disk != disk && latest_log_disk == description->disk)
moveFileBetweenDisks(latest_log_disk, description, disk, description->path);
}
/// Start new log if we don't initialize writer from previous log. All logs can be "complete".
if (!current_writer->isFileSet())
current_writer->rotate(max_log_id + 1);
/// Move files to correct disks
auto latest_start_index = current_writer->getStartIndex();
auto latest_log_disk = getLatestLogDisk();
auto disk = getDisk();
for (const auto & [start_index, description] : existing_changelogs)
{
/// latest log should already be on latest_log_disk
if (start_index == latest_start_index)
{
chassert(description->disk == latest_log_disk);
continue;
}
if (description->disk != disk)
moveFileBetweenDisks(description->disk, description, disk, description->path);
}
initialized = true;
}
void Changelog::initWriter(ChangelogFileDescriptionPtr description)
{
if (description->expectedEntriesCountInLog() != rotate_interval)
LOG_TRACE(
log,
"Looks like rotate_logs_interval was changed, current {}, expected entries in last log {}",
rotate_interval,
description->expectedEntriesCountInLog());
LOG_TRACE(log, "Continue to write into {}", description->path);
auto log_disk = description->disk;
auto latest_log_disk = getLatestLogDisk();
if (log_disk != latest_log_disk)
moveFileBetweenDisks(log_disk, description, latest_log_disk, description->path);
current_writer->setFile(std::move(description), WriteMode::Append);
}
namespace
{
std::string getCurrentTimestampFolder()
{
const auto timestamp = LocalDateTime{std::time(nullptr)};
return fmt::format(
"{:02}{:02}{:02}T{:02}{:02}{:02}",
timestamp.year(),
timestamp.month(),
timestamp.day(),
timestamp.hour(),
timestamp.minute(),
timestamp.second());
std::string getCurrentTimestampFolder()
{
const auto timestamp = LocalDateTime{std::time(nullptr)};
return fmt::format(
"{:02}{:02}{:02}T{:02}{:02}{:02}",
timestamp.year(),
timestamp.month(),
timestamp.day(),
timestamp.hour(),
timestamp.minute(),
timestamp.second());
}
}
DiskPtr Changelog::getDisk() const
{
return keeper_context->getLogDisk();
}
DiskPtr Changelog::getLatestLogDisk() const
{
return keeper_context->getLatestLogDisk();
}
void Changelog::removeExistingLogs(ChangelogIter begin, ChangelogIter end)
{
const auto timestamp_folder = changelogs_detached_dir / getCurrentTimestampFolder();
auto disk = getDisk();
const auto timestamp_folder = (fs::path(changelogs_detached_dir) / getCurrentTimestampFolder()).generic_string();
for (auto itr = begin; itr != end;)
{
if (!std::filesystem::exists(timestamp_folder))
if (!disk->exists(timestamp_folder))
{
LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder.generic_string());
std::filesystem::create_directories(timestamp_folder);
LOG_WARNING(log, "Moving broken logs to {}", timestamp_folder);
disk->createDirectories(timestamp_folder);
}
LOG_WARNING(log, "Removing changelog {}", itr->second->path);
const std::filesystem::path & path = itr->second->path;
const auto new_path = timestamp_folder / path.filename();
std::filesystem::rename(path, new_path);
auto changelog_disk = itr->second->disk;
if (changelog_disk == disk)
{
try
{
disk->moveFile(path.generic_string(), new_path.generic_string());
}
catch (const DB::Exception & e)
{
if (e.code() == DB::ErrorCodes::NOT_IMPLEMENTED)
moveFileBetweenDisks(changelog_disk, itr->second, disk, new_path);
}
}
else
moveFileBetweenDisks(changelog_disk, itr->second, disk, new_path);
itr = existing_changelogs.erase(itr);
}
}
@ -882,7 +1085,6 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before writing records");
{
std::lock_guard lock(writer_mutex);
/// This write_at require to overwrite everything in this file and also in previous file(s)
const bool go_to_previous_file = index < current_writer->getStartIndex();
@ -898,13 +1100,18 @@ void Changelog::writeAt(uint64_t index, const LogEntryPtr & log_entry)
else
description = std::prev(index_changelog)->second;
auto log_disk = description->disk;
auto latest_log_disk = getLatestLogDisk();
if (log_disk != latest_log_disk)
moveFileBetweenDisks(log_disk, description, latest_log_disk, description->path);
current_writer->setFile(std::move(description), WriteMode::Append);
/// Remove all subsequent files if overwritten something in previous one
auto to_remove_itr = existing_changelogs.upper_bound(index);
for (auto itr = to_remove_itr; itr != existing_changelogs.end();)
{
std::filesystem::remove(itr->second->path);
itr->second->disk->removeFile(itr->second->path);
itr = existing_changelogs.erase(itr);
}
}
@ -954,14 +1161,22 @@ void Changelog::compact(uint64_t up_to_log_index)
LOG_INFO(log, "Removing changelog {} because of compaction", changelog_description.path);
/// If failed to push to queue for background removing, then we will remove it now
if (!log_files_to_delete_queue.tryPush(changelog_description.path, 1))
if (!log_files_to_delete_queue.tryPush({changelog_description.path, changelog_description.disk}, 1))
{
std::error_code ec;
std::filesystem::remove(changelog_description.path, ec);
if (ec)
LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", changelog_description.path, ec.message());
else
LOG_INFO(log, "Removed changelog {} because of compaction", changelog_description.path);
try
{
changelog_description.disk->removeFile(changelog_description.path);
LOG_INFO(log, "Removed changelog {} because of compaction.", changelog_description.path);
}
catch (Exception & e)
{
LOG_WARNING(
log, "Failed to remove changelog {} in compaction, error message: {}", changelog_description.path, e.message());
}
catch (...)
{
tryLogCurrentException(log);
}
}
changelog_description.deleted = true;
@ -1151,14 +1366,23 @@ Changelog::~Changelog()
void Changelog::cleanLogThread()
{
std::string path;
while (log_files_to_delete_queue.pop(path))
std::pair<std::string, DiskPtr> path_with_disk;
while (log_files_to_delete_queue.pop(path_with_disk))
{
std::error_code ec;
if (std::filesystem::remove(path, ec))
const auto & [path, disk] = path_with_disk;
try
{
disk->removeFile(path);
LOG_INFO(log, "Removed changelog {} because of compaction.", path);
else
LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", path, ec.message());
}
catch (Exception & e)
{
LOG_WARNING(log, "Failed to remove changelog {} in compaction, error message: {}", path, e.message());
}
catch (...)
{
tryLogCurrentException(log);
}
}
}

View File

@ -11,6 +11,7 @@
#include <libnuraft/raft_server.hxx>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ThreadPool.h>
#include <Coordination/KeeperContext.h>
namespace DB
{
@ -59,6 +60,7 @@ struct ChangelogFileDescription
uint64_t to_log_index;
std::string extension;
DiskPtr disk;
std::string path;
bool deleted = false;
@ -87,9 +89,9 @@ class Changelog
{
public:
Changelog(
const std::string & changelogs_dir_,
Poco::Logger * log_,
LogFileSettings log_file_settings);
LogFileSettings log_file_settings,
KeeperContextPtr keeper_context_);
Changelog(Changelog &&) = delete;
@ -152,6 +154,9 @@ private:
/// Pack log_entry into changelog record
static ChangelogRecord buildRecord(uint64_t index, const LogEntryPtr & log_entry);
DiskPtr getDisk() const;
DiskPtr getLatestLogDisk() const;
/// Currently existing changelogs
std::map<uint64_t, ChangelogFileDescriptionPtr> existing_changelogs;
@ -169,8 +174,7 @@ private:
/// Clean useless log files in a background thread
void cleanLogThread();
const std::filesystem::path changelogs_dir;
const std::filesystem::path changelogs_detached_dir;
const String changelogs_detached_dir;
const uint64_t rotate_interval;
Poco::Logger * log;
@ -185,7 +189,7 @@ private:
uint64_t max_log_id = 0;
/// For compaction, queue of delete not used logs
/// 128 is enough, even if log is not removed, it's not a problem
ConcurrentBoundedQueue<std::string> log_files_to_delete_queue{128};
ConcurrentBoundedQueue<std::pair<std::string, DiskPtr>> log_files_to_delete_queue{128};
ThreadFromGlobalPool clean_log_thread;
struct AppendLog
@ -223,6 +227,8 @@ private:
nuraft::wptr<nuraft::raft_server> raft_server;
KeeperContextPtr keeper_context;
bool initialized = false;
};

View File

@ -85,14 +85,6 @@ void KeeperConfigurationAndSettings::dump(WriteBufferFromOwnString & buf) const
writeText(four_letter_word_allow_list, buf);
buf.write('\n');
writeText("log_storage_path=", buf);
writeText(log_storage_path, buf);
buf.write('\n');
writeText("snapshot_storage_path=", buf);
writeText(snapshot_storage_path, buf);
buf.write('\n');
/// coordination_settings
writeText("max_requests_batch_size=", buf);
@ -188,61 +180,9 @@ KeeperConfigurationAndSettings::loadFromConfig(const Poco::Util::AbstractConfigu
DEFAULT_FOUR_LETTER_WORD_CMD));
ret->log_storage_path = getLogsPathFromConfig(config, standalone_keeper_);
ret->snapshot_storage_path = getSnapshotsPathFromConfig(config, standalone_keeper_);
ret->state_file_path = getStateFilePathFromConfig(config, standalone_keeper_);
ret->coordination_settings->loadFromConfig("keeper_server.coordination_settings", config);
return ret;
}
String KeeperConfigurationAndSettings::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_)
{
/// the most specialized path
if (config.has("keeper_server.log_storage_path"))
return config.getString("keeper_server.log_storage_path");
if (config.has("keeper_server.storage_path"))
return std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs";
if (standalone_keeper_)
return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs";
else
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs";
}
String KeeperConfigurationAndSettings::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_)
{
/// the most specialized path
if (config.has("keeper_server.snapshot_storage_path"))
return config.getString("keeper_server.snapshot_storage_path");
if (config.has("keeper_server.storage_path"))
return std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots";
if (standalone_keeper_)
return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots";
else
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots";
}
String KeeperConfigurationAndSettings::getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_)
{
if (config.has("keeper_server.storage_path"))
return std::filesystem::path{config.getString("keeper_server.storage_path")} / "state";
if (config.has("keeper_server.snapshot_storage_path"))
return std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path() / "state";
if (config.has("keeper_server.log_storage_path"))
return std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path() / "state";
if (standalone_keeper_)
return std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "state";
else
return std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/state";
}
}

View File

@ -82,17 +82,8 @@ struct KeeperConfigurationAndSettings
bool standalone_keeper;
CoordinationSettingsPtr coordination_settings;
String log_storage_path;
String snapshot_storage_path;
String state_file_path;
void dump(WriteBufferFromOwnString & buf) const;
static std::shared_ptr<KeeperConfigurationAndSettings> loadFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
private:
static String getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
static String getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
static String getStateFilePathFromConfig(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper_);
};
using KeeperConfigurationAndSettingsPtr = std::shared_ptr<KeeperConfigurationAndSettings>;

View File

@ -297,6 +297,7 @@ String ConfCommand::run()
StringBuffer buf;
keeper_dispatcher.getKeeperConfigurationAndSettings()->dump(buf);
keeper_dispatcher.getKeeperContext()->dumpConfiguration(buf);
return buf.str();
}
@ -542,7 +543,7 @@ String CleanResourcesCommand::run()
String FeatureFlagsCommand::run()
{
const auto & feature_flags = keeper_dispatcher.getKeeperContext()->feature_flags;
const auto & feature_flags = keeper_dispatcher.getKeeperContext()->getFeatureFlags();
StringBuffer ret;

View File

@ -28,8 +28,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM
size_t zxid = 0;
size_t session_with_watches = 0;
size_t paths_watched = 0;
size_t snapshot_dir_size = 0;
size_t log_dir_size = 0;
//size_t snapshot_dir_size = 0;
//size_t log_dir_size = 0;
if (keeper_dispatcher.isServerActive())
{
@ -49,8 +49,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM
latest_snapshot_size = state_machine.getLatestSnapshotBufSize();
session_with_watches = state_machine.getSessionsWithWatchesCount();
paths_watched = state_machine.getWatchedPathsCount();
snapshot_dir_size = keeper_dispatcher.getSnapDirSize();
log_dir_size = keeper_dispatcher.getLogDirSize();
//snapshot_dir_size = keeper_dispatcher.getSnapDirSize();
//log_dir_size = keeper_dispatcher.getLogDirSize();
# if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount();
@ -85,8 +85,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM
new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." };
new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." };
new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." };
new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." };
new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." };
//new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." };
//new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." };
auto keeper_log_info = keeper_dispatcher.getKeeperLogInfo();
@ -108,8 +108,8 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM
}
KeeperAsynchronousMetrics::KeeperAsynchronousMetrics(
TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), tiny_context(std::move(tiny_context_))
ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
: AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_))
{
}
@ -117,7 +117,7 @@ void KeeperAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values
{
#if USE_NURAFT
{
auto keeper_dispatcher = tiny_context->tryGetKeeperDispatcher();
auto keeper_dispatcher = context->tryGetKeeperDispatcher();
if (keeper_dispatcher)
updateKeeperInformation(*keeper_dispatcher, new_values);
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <Coordination/TinyContext.h>
#include <Interpreters/Context.h>
#include <Common/AsynchronousMetrics.h>
namespace DB
@ -13,10 +13,10 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics
{
public:
KeeperAsynchronousMetrics(
TinyContextPtr tiny_context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
ContextPtr context_, int update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
private:
TinyContextPtr tiny_context;
ContextPtr context;
void updateImpl(AsynchronousMetricValues & new_values, TimePoint update_time, TimePoint current_time) override;
};

View File

@ -1,4 +1,9 @@
#include <Coordination/KeeperContext.h>
#include <Coordination/Defines.h>
#include <Disks/DiskLocal.h>
#include <Interpreters/Context.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Coordination/KeeperConstants.h>
#include <Common/logger_useful.h>
#include <Coordination/KeeperFeatureFlags.h>
@ -14,14 +19,15 @@ extern const int BAD_ARGUMENTS;
}
KeeperContext::KeeperContext()
KeeperContext::KeeperContext(bool standalone_keeper_)
: disk_selector(std::make_shared<DiskSelector>())
, standalone_keeper(standalone_keeper_)
{
/// enable by default some feature flags
feature_flags.enableFeatureFlag(KeeperFeatureFlag::FILTERED_LIST);
feature_flags.enableFeatureFlag(KeeperFeatureFlag::MULTI_READ);
system_nodes_with_data[keeper_api_feature_flags_path] = feature_flags.getFeatureFlags();
/// for older clients, the default is equivalent to WITH_MULTI_READ version
system_nodes_with_data[keeper_api_version_path] = toString(static_cast<uint8_t>(KeeperApiVersion::WITH_MULTI_READ));
}
@ -31,6 +37,264 @@ void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config)
digest_enabled = config.getBool("keeper_server.digest_enabled", false);
ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false);
initializeFeatureFlags(config);
initializeDisks(config);
}
void KeeperContext::initializeDisks(const Poco::Util::AbstractConfiguration & config)
{
disk_selector->initialize(config, "storage_configuration.disks", Context::getGlobalContextInstance());
log_storage = getLogsPathFromConfig(config);
if (config.has("keeper_server.latest_log_storage_disk"))
latest_log_storage = config.getString("keeper_server.latest_log_storage_disk");
else
latest_log_storage = log_storage;
const auto collect_old_disk_names = [&](const std::string_view key_prefix, std::vector<std::string> & disk_names)
{
Poco::Util::AbstractConfiguration::Keys disk_name_keys;
config.keys("keeper_server", disk_name_keys);
for (const auto & key : disk_name_keys)
{
if (key.starts_with(key_prefix))
disk_names.push_back(config.getString(fmt::format("keeper_server.{}", key)));
}
};
collect_old_disk_names("old_log_storage_disk", old_log_disk_names);
collect_old_disk_names("old_snapshot_storage_disk", old_snapshot_disk_names);
snapshot_storage = getSnapshotsPathFromConfig(config);
if (config.has("keeper_server.latest_snapshot_storage_disk"))
latest_snapshot_storage = config.getString("keeper_server.latest_snapshot_storage_disk");
else
latest_snapshot_storage = snapshot_storage;
state_file_storage = getStatePathFromConfig(config);
}
KeeperContext::Phase KeeperContext::getServerState() const
{
return server_state;
}
void KeeperContext::setServerState(KeeperContext::Phase server_state_)
{
server_state = server_state_;
}
bool KeeperContext::ignoreSystemPathOnStartup() const
{
return ignore_system_path_on_startup;
}
bool KeeperContext::digestEnabled() const
{
return digest_enabled;
}
void KeeperContext::setDigestEnabled(bool digest_enabled_)
{
digest_enabled = digest_enabled_;
}
DiskPtr KeeperContext::getDisk(const Storage & storage) const
{
if (const auto * storage_disk = std::get_if<DiskPtr>(&storage))
return *storage_disk;
const auto & disk_name = std::get<std::string>(storage);
return disk_selector->get(disk_name);
}
DiskPtr KeeperContext::getLogDisk() const
{
return getDisk(log_storage);
}
std::vector<DiskPtr> KeeperContext::getOldLogDisks() const
{
std::vector<DiskPtr> old_log_disks;
old_log_disks.reserve(old_log_disk_names.size());
for (const auto & disk_name : old_log_disk_names)
old_log_disks.push_back(disk_selector->get(disk_name));
return old_log_disks;
}
DiskPtr KeeperContext::getLatestLogDisk() const
{
return getDisk(latest_log_storage);
}
void KeeperContext::setLogDisk(DiskPtr disk)
{
log_storage = disk;
latest_log_storage = std::move(disk);
}
DiskPtr KeeperContext::getLatestSnapshotDisk() const
{
return getDisk(latest_snapshot_storage);
}
DiskPtr KeeperContext::getSnapshotDisk() const
{
return getDisk(snapshot_storage);
}
std::vector<DiskPtr> KeeperContext::getOldSnapshotDisks() const
{
std::vector<DiskPtr> old_snapshot_disks;
old_snapshot_disks.reserve(old_snapshot_disk_names.size());
for (const auto & disk_name : old_snapshot_disk_names)
old_snapshot_disks.push_back(disk_selector->get(disk_name));
return old_snapshot_disks;
}
void KeeperContext::setSnapshotDisk(DiskPtr disk)
{
snapshot_storage = std::move(disk);
latest_snapshot_storage = snapshot_storage;
}
DiskPtr KeeperContext::getStateFileDisk() const
{
return getDisk(state_file_storage);
}
void KeeperContext::setStateFileDisk(DiskPtr disk)
{
state_file_storage = std::move(disk);
}
const std::unordered_map<std::string, std::string> & KeeperContext::getSystemNodesWithData() const
{
return system_nodes_with_data;
}
const KeeperFeatureFlags & KeeperContext::getFeatureFlags() const
{
return feature_flags;
}
void KeeperContext::dumpConfiguration(WriteBufferFromOwnString & buf) const
{
auto dump_disk_info = [&](const std::string_view prefix, const IDisk & disk)
{
writeText(fmt::format("{}_path=", prefix), buf);
writeText(disk.getPath(), buf);
buf.write('\n');
writeText(fmt::format("{}_disk=", prefix), buf);
writeText(disk.getName(), buf);
buf.write('\n');
};
{
auto log_disk = getDisk(log_storage);
dump_disk_info("log_storage", *log_disk);
auto latest_log_disk = getDisk(latest_log_storage);
if (log_disk != latest_log_disk)
dump_disk_info("latest_log_storage", *latest_log_disk);
}
{
auto snapshot_disk = getDisk(snapshot_storage);
dump_disk_info("snapshot_storage", *snapshot_disk);
}
}
KeeperContext::Storage KeeperContext::getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const
{
const auto create_local_disk = [](const auto & path)
{
if (!fs::exists(path))
fs::create_directories(path);
return std::make_shared<DiskLocal>("LocalLogDisk", path, 0);
};
/// the most specialized path
if (config.has("keeper_server.log_storage_path"))
return create_local_disk(config.getString("keeper_server.log_storage_path"));
if (config.has("keeper_server.log_storage_disk"))
return config.getString("keeper_server.log_storage_disk");
if (config.has("keeper_server.storage_path"))
return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "logs");
if (standalone_keeper)
return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "logs");
else
return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/logs");
}
KeeperContext::Storage KeeperContext::getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const
{
const auto create_local_disk = [](const auto & path)
{
if (!fs::exists(path))
fs::create_directories(path);
return std::make_shared<DiskLocal>("LocalSnapshotDisk", path, 0);
};
/// the most specialized path
if (config.has("keeper_server.snapshot_storage_path"))
return create_local_disk(config.getString("keeper_server.snapshot_storage_path"));
if (config.has("keeper_server.snapshot_storage_disk"))
return config.getString("keeper_server.snapshot_storage_disk");
if (config.has("keeper_server.storage_path"))
return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")} / "snapshots");
if (standalone_keeper)
return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)} / "snapshots");
else
return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination/snapshots");
}
KeeperContext::Storage KeeperContext::getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const
{
const auto create_local_disk = [](const auto & path)
{
if (!fs::exists(path))
fs::create_directories(path);
return std::make_shared<DiskLocal>("LocalStateFileDisk", path, 0);
};
if (config.has("keeper_server.state_storage_disk"))
return config.getString("keeper_server.state_storage_disk");
if (config.has("keeper_server.storage_path"))
return create_local_disk(std::filesystem::path{config.getString("keeper_server.storage_path")});
if (config.has("keeper_server.snapshot_storage_path"))
return create_local_disk(std::filesystem::path(config.getString("keeper_server.snapshot_storage_path")).parent_path());
if (config.has("keeper_server.log_storage_path"))
return create_local_disk(std::filesystem::path(config.getString("keeper_server.log_storage_path")).parent_path());
if (standalone_keeper)
return create_local_disk(std::filesystem::path{config.getString("path", KEEPER_DEFAULT_PATH)});
else
return create_local_disk(std::filesystem::path{config.getString("path", DBMS_DEFAULT_PATH)} / "coordination");
}
void KeeperContext::initializeFeatureFlags(const Poco::Util::AbstractConfiguration & config)
{
static const std::string feature_flags_key = "keeper_server.feature_flags";
if (config.has(feature_flags_key))
{

View File

@ -1,16 +1,21 @@
#pragma once
#include <Poco/Util/AbstractConfiguration.h>
#include <Coordination/KeeperFeatureFlags.h>
#include <IO/WriteBufferFromString.h>
#include <Disks/DiskSelector.h>
#include <cstdint>
#include <memory>
namespace DB
{
struct KeeperContext
class KeeperContext
{
KeeperContext();
void initialize(const Poco::Util::AbstractConfiguration & config);
public:
explicit KeeperContext(bool standalone_keeper_);
enum class Phase : uint8_t
{
@ -19,11 +24,64 @@ struct KeeperContext
SHUTDOWN
};
void initialize(const Poco::Util::AbstractConfiguration & config);
Phase getServerState() const;
void setServerState(Phase server_state_);
bool ignoreSystemPathOnStartup() const;
bool digestEnabled() const;
void setDigestEnabled(bool digest_enabled_);
DiskPtr getLatestLogDisk() const;
DiskPtr getLogDisk() const;
std::vector<DiskPtr> getOldLogDisks() const;
void setLogDisk(DiskPtr disk);
DiskPtr getLatestSnapshotDisk() const;
DiskPtr getSnapshotDisk() const;
std::vector<DiskPtr> getOldSnapshotDisks() const;
void setSnapshotDisk(DiskPtr disk);
DiskPtr getStateFileDisk() const;
void setStateFileDisk(DiskPtr disk);
const std::unordered_map<std::string, std::string> & getSystemNodesWithData() const;
const KeeperFeatureFlags & getFeatureFlags() const;
void dumpConfiguration(WriteBufferFromOwnString & buf) const;
private:
/// local disk defined using path or disk name
using Storage = std::variant<DiskPtr, std::string>;
void initializeFeatureFlags(const Poco::Util::AbstractConfiguration & config);
void initializeDisks(const Poco::Util::AbstractConfiguration & config);
Storage getLogsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
Storage getSnapshotsPathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
Storage getStatePathFromConfig(const Poco::Util::AbstractConfiguration & config) const;
DiskPtr getDisk(const Storage & storage) const;
Phase server_state{Phase::INIT};
bool ignore_system_path_on_startup{false};
bool digest_enabled{true};
std::shared_ptr<DiskSelector> disk_selector;
Storage log_storage;
Storage latest_log_storage;
Storage snapshot_storage;
Storage latest_snapshot_storage;
Storage state_file_storage;
std::vector<std::string> old_log_disk_names;
std::vector<std::string> old_snapshot_disk_names;
bool standalone_keeper;
std::unordered_map<std::string, std::string> system_nodes_with_data;
KeeperFeatureFlags feature_flags;

View File

@ -38,8 +38,6 @@ namespace ProfileEvents
extern const Event MemoryAllocatorPurgeTimeMicroseconds;
}
namespace fs = std::filesystem;
namespace DB
{
@ -238,13 +236,13 @@ void KeeperDispatcher::snapshotThread()
try
{
auto snapshot_path = task.create_snapshot(std::move(task.snapshot));
auto snapshot_file_info = task.create_snapshot(std::move(task.snapshot));
if (snapshot_path.empty())
if (snapshot_file_info.path.empty())
continue;
if (isLeader())
snapshot_s3.uploadSnapshot(snapshot_path);
snapshot_s3.uploadSnapshot(snapshot_file_info);
}
catch (...)
{
@ -336,7 +334,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
snapshot_s3.startup(config, macros);
keeper_context = std::make_shared<KeeperContext>();
keeper_context = std::make_shared<KeeperContext>(standalone_keeper);
keeper_context->initialize(config);
server = std::make_unique<KeeperServer>(
@ -777,35 +775,37 @@ void KeeperDispatcher::updateKeeperStatLatency(uint64_t process_time_ms)
keeper_stats.updateLatency(process_time_ms);
}
static uint64_t getDirSize(const fs::path & dir)
static uint64_t getTotalSize(const DiskPtr & disk, const std::string & path = "")
{
checkStackSize();
if (!fs::exists(dir))
return 0;
fs::directory_iterator it(dir);
fs::directory_iterator end;
uint64_t size{0};
while (it != end)
uint64_t size = 0;
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
{
if (it->is_regular_file())
size += fs::file_size(*it);
if (disk->isFile(it->path()))
size += disk->getFileSize(it->path());
else
size += getDirSize(it->path());
++it;
size += getTotalSize(disk, it->path());
}
return size;
}
uint64_t KeeperDispatcher::getLogDirSize() const
{
return getDirSize(configuration_and_settings->log_storage_path);
auto log_disk = keeper_context->getLogDisk();
auto size = getTotalSize(log_disk);
auto latest_log_disk = keeper_context->getLatestLogDisk();
if (log_disk != latest_log_disk)
size += getTotalSize(latest_log_disk);
return size;
}
uint64_t KeeperDispatcher::getSnapDirSize() const
{
return getDirSize(configuration_and_settings->snapshot_storage_path);
return getTotalSize(keeper_context->getSnapshotDisk());
}
Keeper4LWInfo KeeperDispatcher::getKeeper4LWInfo() const

View File

@ -205,7 +205,6 @@ public:
return keeper_context;
}
void incrementPacketsSent()
{
keeper_stats.incrementPacketsSent();

View File

@ -1,14 +1,14 @@
#include <Coordination/KeeperLogStore.h>
#include <IO/CompressionMethod.h>
#include <Disks/DiskLocal.h>
#include <Common/logger_useful.h>
namespace DB
{
KeeperLogStore::KeeperLogStore(
const std::string & changelogs_path, LogFileSettings log_file_settings)
KeeperLogStore::KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context)
: log(&Poco::Logger::get("KeeperLogStore"))
, changelog(changelogs_path, log, log_file_settings)
, changelog(log, log_file_settings, keeper_context)
{
if (log_file_settings.force_sync)
LOG_INFO(log, "force_sync enabled");

View File

@ -4,6 +4,7 @@
#include <mutex>
#include <Core/Types.h>
#include <Coordination/Changelog.h>
#include <Coordination/KeeperContext.h>
#include <base/defines.h>
namespace DB
@ -13,7 +14,7 @@ namespace DB
class KeeperLogStore : public nuraft::log_store
{
public:
KeeperLogStore(const std::string & changelogs_path, LogFileSettings log_file_settings);
KeeperLogStore(LogFileSettings log_file_settings, KeeperContextPtr keeper_context);
/// Read log storage from filesystem starting from last_commited_log_index
void init(uint64_t last_commited_log_index, uint64_t logs_to_keep);

View File

@ -26,6 +26,7 @@
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Common/Stopwatch.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Disks/DiskLocal.h>
namespace DB
{
@ -124,7 +125,6 @@ KeeperServer::KeeperServer(
state_machine = nuraft::cs_new<KeeperStateMachine>(
responses_queue_,
snapshots_queue_,
configuration_and_settings_->snapshot_storage_path,
coordination_settings,
keeper_context,
config.getBool("keeper_server.upload_snapshot_on_exit", true) ? &snapshot_manager_s3 : nullptr,
@ -134,10 +134,10 @@ KeeperServer::KeeperServer(
state_manager = nuraft::cs_new<KeeperStateManager>(
server_id,
"keeper_server",
configuration_and_settings_->log_storage_path,
configuration_and_settings_->state_file_path,
"state",
config,
coordination_settings);
coordination_settings,
keeper_context);
}
/**
@ -413,7 +413,7 @@ void KeeperServer::startup(const Poco::Util::AbstractConfiguration & config, boo
launchRaftServer(config, enable_ipv6);
keeper_context->server_state = KeeperContext::Phase::RUNNING;
keeper_context->setServerState(KeeperContext::Phase::RUNNING);
}
void KeeperServer::shutdownRaftServer()
@ -428,7 +428,7 @@ void KeeperServer::shutdownRaftServer()
raft_instance->shutdown();
keeper_context->server_state = KeeperContext::Phase::SHUTDOWN;
keeper_context->setServerState(KeeperContext::Phase::SHUTDOWN);
if (create_snapshot_on_exit)
raft_instance->create_snapshot();

View File

@ -9,13 +9,15 @@
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Coordination/pathUtils.h>
#include <filesystem>
#include <memory>
#include <Common/logger_useful.h>
#include <Coordination/KeeperContext.h>
#include <Coordination/pathUtils.h>
#include <Coordination/KeeperConstants.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include "Core/Field.h"
#include <Disks/DiskLocal.h>
namespace DB
@ -30,6 +32,25 @@ namespace ErrorCodes
namespace
{
constexpr std::string_view tmp_prefix = "tmp_";
void moveFileBetweenDisks(DiskPtr disk_from, const std::string & path_from, DiskPtr disk_to, const std::string & path_to)
{
/// we use empty file with prefix tmp_ to detect incomplete copies
/// if a copy is complete we don't care from which disk we use the same file
/// so it's okay if a failure happens after removing of tmp file but before we remove
/// the snapshot from the source disk
auto from_path = fs::path(path_from);
auto tmp_snapshot_name = from_path.parent_path() / (std::string{tmp_prefix} + from_path.filename().string());
{
auto buf = disk_to->writeFile(tmp_snapshot_name);
buf->finalize();
}
disk_from->copyFile(from_path, *disk_to, path_to, {});
disk_to->removeFile(tmp_snapshot_name);
disk_from->removeFile(path_from);
}
uint64_t getSnapshotPathUpToLogIdx(const String & snapshot_path)
{
std::filesystem::path path(snapshot_path);
@ -41,7 +62,7 @@ namespace
std::string getSnapshotFileName(uint64_t up_to_log_idx, bool compress_zstd)
{
auto base = std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin";
auto base = fmt::format("snapshot_{}.bin", up_to_log_idx);
if (compress_zstd)
base += ".zstd";
return base;
@ -156,7 +177,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
if (snapshot.version >= SnapshotVersion::V5)
{
writeBinary(snapshot.zxid, out);
if (keeper_context->digest_enabled)
if (keeper_context->digestEnabled())
{
writeBinary(static_cast<uint8_t>(KeeperStorage::CURRENT_DIGEST_VERSION), out);
writeBinary(snapshot.nodes_digest, out);
@ -185,7 +206,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
}
/// Serialize data tree
writeBinary(snapshot.snapshot_container_size - keeper_context->system_nodes_with_data.size(), out);
writeBinary(snapshot.snapshot_container_size - keeper_context->getSystemNodesWithData().size(), out);
size_t counter = 0;
for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter)
{
@ -267,7 +288,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in);
KeeperStorage & storage = *deserialization_result.storage;
bool recalculate_digest = keeper_context->digest_enabled;
bool recalculate_digest = keeper_context->digestEnabled();
if (version >= SnapshotVersion::V5)
{
readBinary(storage.zxid, in);
@ -349,7 +370,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
const std::string error_msg = fmt::format("Cannot read node on path {} from a snapshot because it is used as a system node", path);
if (match_result == IS_CHILD)
{
if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT)
if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT)
{
LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg);
continue;
@ -365,7 +386,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
{
if (!is_node_empty(node))
{
if (keeper_context->ignore_system_path_on_startup || keeper_context->server_state != KeeperContext::Phase::INIT)
if (keeper_context->ignoreSystemPathOnStartup() || keeper_context->getServerState() != KeeperContext::Phase::INIT)
{
LOG_ERROR(&Poco::Logger::get("KeeperSnapshotManager"), "{}. Ignoring it", error_msg);
node = KeeperStorage::Node{};
@ -394,9 +415,9 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial
{
if (itr.key != "/")
{
auto parent_path = parentPath(itr.key);
auto parent_path = parentNodePath(itr.key);
storage.container.updateValue(
parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseName(path), /*update_size*/ version < SnapshotVersion::V4); });
parent_path, [version, path = itr.key](KeeperStorage::Node & value) { value.addChild(getBaseNodeName(path), /*update_size*/ version < SnapshotVersion::V4); });
}
}
@ -507,70 +528,110 @@ KeeperStorageSnapshot::~KeeperStorageSnapshot()
}
KeeperSnapshotManager::KeeperSnapshotManager(
const std::string & snapshots_path_,
size_t snapshots_to_keep_,
const KeeperContextPtr & keeper_context_,
bool compress_snapshots_zstd_,
const std::string & superdigest_,
size_t storage_tick_time_)
: snapshots_path(snapshots_path_)
, snapshots_to_keep(snapshots_to_keep_)
: snapshots_to_keep(snapshots_to_keep_)
, compress_snapshots_zstd(compress_snapshots_zstd_)
, superdigest(superdigest_)
, storage_tick_time(storage_tick_time_)
, keeper_context(keeper_context_)
{
namespace fs = std::filesystem;
if (!fs::exists(snapshots_path))
fs::create_directories(snapshots_path);
for (const auto & p : fs::directory_iterator(snapshots_path))
const auto load_snapshot_from_disk = [&](const auto & disk)
{
const auto & path = p.path();
LOG_TRACE(log, "Reading from disk {}", disk->getName());
std::unordered_map<std::string, std::string> incomplete_files;
if (!path.has_filename())
continue;
if (startsWith(path.filename(), "tmp_")) /// Unfinished tmp files
const auto clean_incomplete_file = [&](const auto & file_path)
{
std::filesystem::remove(p);
continue;
if (auto incomplete_it = incomplete_files.find(fs::path(file_path).filename()); incomplete_it != incomplete_files.end())
{
LOG_TRACE(log, "Removing {} from {}", file_path, disk->getName());
disk->removeFile(file_path);
disk->removeFile(incomplete_it->second);
incomplete_files.erase(incomplete_it);
return true;
}
return false;
};
std::vector<std::string> snapshot_files;
for (auto it = disk->iterateDirectory(""); it->isValid(); it->next())
{
if (it->name().starts_with(tmp_prefix))
{
incomplete_files.emplace(it->name().substr(tmp_prefix.size()), it->path());
continue;
}
if (it->name().starts_with("snapshot_") && !clean_incomplete_file(it->path()))
snapshot_files.push_back(it->path());
}
/// Not snapshot file
if (!startsWith(path.filename(), "snapshot_"))
for (const auto & snapshot_file : snapshot_files)
{
continue;
if (clean_incomplete_file(fs::path(snapshot_file).filename()))
continue;
LOG_TRACE(log, "Found {} on {}", snapshot_file, disk->getName());
size_t snapshot_up_to = getSnapshotPathUpToLogIdx(snapshot_file);
auto [_, inserted] = existing_snapshots.insert_or_assign(snapshot_up_to, SnapshotFileInfo{snapshot_file, disk});
if (!inserted)
LOG_WARNING(
&Poco::Logger::get("KeeperSnapshotManager"),
"Found another snapshots with last log idx {}, will use snapshot from disk {}",
snapshot_up_to,
disk->getName());
}
size_t snapshot_up_to = getSnapshotPathUpToLogIdx(p.path());
existing_snapshots[snapshot_up_to] = p.path();
}
for (const auto & [name, path] : incomplete_files)
disk->removeFile(path);
};
for (const auto & disk : keeper_context->getOldSnapshotDisks())
load_snapshot_from_disk(disk);
auto disk = getDisk();
load_snapshot_from_disk(disk);
auto latest_snapshot_disk = getLatestSnapshotDisk();
if (latest_snapshot_disk != disk)
load_snapshot_from_disk(latest_snapshot_disk);
removeOutdatedSnapshotsIfNeeded();
moveSnapshotsIfNeeded();
}
std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx)
{
ReadBufferFromNuraftBuffer reader(buffer);
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd);
auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name;
std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name;
std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name;
WriteBufferFromFile plain_buf(tmp_snapshot_path);
copyData(reader, plain_buf);
plain_buf.sync();
auto disk = getLatestSnapshotDisk();
std::filesystem::rename(tmp_snapshot_path, new_snapshot_path);
{
auto buf = disk->writeFile(tmp_snapshot_file_name);
buf->finalize();
}
existing_snapshots.emplace(up_to_log_idx, new_snapshot_path);
auto plain_buf = disk->writeFile(snapshot_file_name);
copyData(reader, *plain_buf);
plain_buf->sync();
plain_buf->finalize();
disk->removeFile(tmp_snapshot_file_name);
existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk});
removeOutdatedSnapshotsIfNeeded();
moveSnapshotsIfNeeded();
return new_snapshot_path;
return {snapshot_file_name, disk};
}
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
@ -584,7 +645,8 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBuff
}
catch (const DB::Exception &)
{
std::filesystem::remove(latest_itr->second);
const auto & [path, disk] = latest_itr->second;
disk->removeFile(path);
existing_snapshots.erase(latest_itr->first);
tryLogCurrentException(__PRETTY_FUNCTION__);
}
@ -595,10 +657,10 @@ nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBuff
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const
{
const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx);
const auto & [snapshot_path, snapshot_disk] = existing_snapshots.at(up_to_log_idx);
WriteBufferFromNuraftBuffer writer;
ReadBufferFromFile reader(snapshot_path);
copyData(reader, writer);
auto reader = snapshot_disk->readFile(snapshot_path);
copyData(*reader, writer);
return writer.getBuffer();
}
@ -659,30 +721,75 @@ SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot()
return deserializeSnapshotFromBuffer(buffer);
}
DiskPtr KeeperSnapshotManager::getDisk() const
{
return keeper_context->getSnapshotDisk();
}
DiskPtr KeeperSnapshotManager::getLatestSnapshotDisk() const
{
return keeper_context->getLatestSnapshotDisk();
}
void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
{
while (existing_snapshots.size() > snapshots_to_keep)
removeSnapshot(existing_snapshots.begin()->first);
}
void KeeperSnapshotManager::moveSnapshotsIfNeeded()
{
/// move snapshots to correct disks
auto disk = getDisk();
auto latest_snapshot_disk = getLatestSnapshotDisk();
auto latest_snapshot_idx = getLatestSnapshotIndex();
for (auto & [idx, file_info] : existing_snapshots)
{
if (idx == latest_snapshot_idx)
{
if (file_info.disk != latest_snapshot_disk)
{
moveFileBetweenDisks(file_info.disk, file_info.path, latest_snapshot_disk, file_info.path);
file_info.disk = latest_snapshot_disk;
}
}
else
{
if (file_info.disk != disk)
{
moveFileBetweenDisks(file_info.disk, file_info.path, disk, file_info.path);
file_info.disk = disk;
}
}
}
}
void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx)
{
auto itr = existing_snapshots.find(log_idx);
if (itr == existing_snapshots.end())
throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx);
std::filesystem::remove(itr->second);
const auto & [path, disk] = itr->second;
disk->removeFile(path);
existing_snapshots.erase(itr);
}
std::pair<std::string, std::error_code> KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot)
SnapshotFileInfo KeeperSnapshotManager::serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot)
{
auto up_to_log_idx = snapshot.snapshot_meta->get_last_log_idx();
auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd);
auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name;
std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name;
std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name;
auto writer = std::make_unique<WriteBufferFromFile>(tmp_snapshot_path, O_WRONLY | O_TRUNC | O_CREAT | O_CLOEXEC | O_APPEND);
auto disk = getLatestSnapshotDisk();
{
auto buf = disk->writeFile(tmp_snapshot_file_name);
buf->finalize();
}
auto writer = disk->writeFile(snapshot_file_name);
std::unique_ptr<WriteBuffer> compressed_writer;
if (compress_snapshots_zstd)
compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3);
@ -693,14 +800,13 @@ std::pair<std::string, std::error_code> KeeperSnapshotManager::serializeSnapshot
compressed_writer->finalize();
compressed_writer->sync();
std::error_code ec;
std::filesystem::rename(tmp_snapshot_path, new_snapshot_path, ec);
if (!ec)
{
existing_snapshots.emplace(up_to_log_idx, new_snapshot_path);
removeOutdatedSnapshotsIfNeeded();
}
return {new_snapshot_path, ec};
disk->removeFile(tmp_snapshot_file_name);
existing_snapshots.emplace(up_to_log_idx, SnapshotFileInfo{snapshot_file_name, disk});
removeOutdatedSnapshotsIfNeeded();
moveSnapshotsIfNeeded();
return {snapshot_file_name, disk};
}
}

View File

@ -6,6 +6,7 @@
#include <IO/WriteBuffer.h>
#include <libnuraft/nuraft.hxx>
#include <Coordination/KeeperContext.h>
#include <Disks/IDisk.h>
namespace DB
{
@ -86,8 +87,14 @@ public:
uint64_t nodes_digest;
};
struct SnapshotFileInfo
{
std::string path;
DiskPtr disk;
};
using KeeperStorageSnapshotPtr = std::shared_ptr<KeeperStorageSnapshot>;
using CreateSnapshotCallback = std::function<std::string(KeeperStorageSnapshotPtr &&)>;
using CreateSnapshotCallback = std::function<SnapshotFileInfo(KeeperStorageSnapshotPtr &&)>;
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
@ -98,7 +105,6 @@ class KeeperSnapshotManager
{
public:
KeeperSnapshotManager(
const std::string & snapshots_path_,
size_t snapshots_to_keep_,
const KeeperContextPtr & keeper_context_,
bool compress_snapshots_zstd_ = true,
@ -112,10 +118,10 @@ public:
nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const;
/// Serialize already compressed snapshot to disk (return path)
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
SnapshotFileInfo serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx);
/// Serialize snapshot directly to disk
std::pair<std::string, std::error_code> serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot);
SnapshotFileInfo serializeSnapshotToDisk(const KeeperStorageSnapshot & snapshot);
SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
@ -139,30 +145,39 @@ public:
return 0;
}
std::string getLatestSnapshotPath() const
SnapshotFileInfo getLatestSnapshotInfo() const
{
if (!existing_snapshots.empty())
{
const auto & path = existing_snapshots.at(getLatestSnapshotIndex());
std::error_code ec;
if (std::filesystem::exists(path, ec))
return path;
const auto & [path, disk] = existing_snapshots.at(getLatestSnapshotIndex());
try
{
if (disk->exists(path))
return {path, disk};
}
catch (...)
{
}
}
return "";
return {"", nullptr};
}
private:
void removeOutdatedSnapshotsIfNeeded();
void moveSnapshotsIfNeeded();
DiskPtr getDisk() const;
DiskPtr getLatestSnapshotDisk() const;
/// Checks first 4 buffer bytes to became sure that snapshot compressed with
/// ZSTD codec.
static bool isZstdCompressed(nuraft::ptr<nuraft::buffer> buffer);
const std::string snapshots_path;
/// How many snapshots to keep before remove
const size_t snapshots_to_keep;
/// All existing snapshots in our path (log_index -> path)
std::map<uint64_t, std::string> existing_snapshots;
std::map<uint64_t, SnapshotFileInfo> existing_snapshots;
/// Compress snapshots in common ZSTD format instead of custom ClickHouse block LZ4 format
const bool compress_snapshots_zstd;
/// Superdigest for deserialization of storage
@ -171,6 +186,8 @@ private:
size_t storage_tick_time;
KeeperContextPtr keeper_context;
Poco::Logger * log = &Poco::Logger::get("KeeperSnapshotManager");
};
/// Keeper create snapshots in background thread. KeeperStateMachine just create

View File

@ -132,8 +132,9 @@ std::shared_ptr<KeeperSnapshotManagerS3::S3Configuration> KeeperSnapshotManagerS
return snapshot_s3_client;
}
void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_path)
void KeeperSnapshotManagerS3::uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info)
{
const auto & [snapshot_path, snapshot_disk] = snapshot_file_info;
try
{
auto s3_client = getSnapshotS3Client();
@ -154,8 +155,9 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa
};
};
LOG_INFO(log, "Will try to upload snapshot on {} to S3", snapshot_path);
ReadBufferFromFile snapshot_file(snapshot_path);
LOG_INFO(log, "Will try to upload snapshot on {} to S3", snapshot_file_info.path);
auto snapshot_file = snapshot_disk->readFile(snapshot_file_info.path);
auto snapshot_name = fs::path(snapshot_path).filename().string();
auto lock_file = fmt::format(".{}_LOCK", snapshot_name);
@ -222,7 +224,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa
});
WriteBufferFromS3 snapshot_writer = create_writer(snapshot_name);
copyData(snapshot_file, snapshot_writer);
copyData(*snapshot_file, snapshot_writer);
snapshot_writer.finalize();
LOG_INFO(log, "Successfully uploaded {} to S3", snapshot_path);
@ -240,31 +242,31 @@ void KeeperSnapshotManagerS3::snapshotS3Thread()
while (!shutdown_called)
{
std::string snapshot_path;
if (!snapshots_s3_queue.pop(snapshot_path))
SnapshotFileInfo snapshot_file_info;
if (!snapshots_s3_queue.pop(snapshot_file_info))
break;
if (shutdown_called)
break;
uploadSnapshotImpl(snapshot_path);
uploadSnapshotImpl(snapshot_file_info);
}
}
void KeeperSnapshotManagerS3::uploadSnapshot(const std::string & path, bool async_upload)
void KeeperSnapshotManagerS3::uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload)
{
if (getSnapshotS3Client() == nullptr)
return;
if (async_upload)
{
if (!snapshots_s3_queue.push(path))
LOG_WARNING(log, "Failed to add snapshot {} to S3 queue", path);
if (!snapshots_s3_queue.push(file_info))
LOG_WARNING(log, "Failed to add snapshot {} to S3 queue", file_info.path);
return;
}
uploadSnapshotImpl(path);
uploadSnapshotImpl(file_info);
}
void KeeperSnapshotManagerS3::startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion<Macros>::Version & macros)

View File

@ -6,10 +6,13 @@
#include <Common/MultiVersion.h>
#include <Common/Macros.h>
#include <Coordination/KeeperSnapshotManager.h>
#if USE_AWS_S3
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/ThreadPool.h>
#include <string>
#endif
@ -24,13 +27,13 @@ public:
/// 'macros' are used to substitute macros in endpoint of disks
void updateS3Configuration(const Poco::Util::AbstractConfiguration & config, const MultiVersion<Macros>::Version & macros);
void uploadSnapshot(const std::string & path, bool async_upload = true);
void uploadSnapshot(const SnapshotFileInfo & file_info, bool async_upload = true);
/// 'macros' are used to substitute macros in endpoint of disks
void startup(const Poco::Util::AbstractConfiguration & config, const MultiVersion<Macros>::Version & macros);
void shutdown();
private:
using SnapshotS3Queue = ConcurrentBoundedQueue<std::string>;
using SnapshotS3Queue = ConcurrentBoundedQueue<SnapshotFileInfo>;
SnapshotS3Queue snapshots_s3_queue;
/// Upload new snapshots to S3
@ -48,7 +51,7 @@ private:
std::shared_ptr<S3Configuration> getSnapshotS3Client() const;
void uploadSnapshotImpl(const std::string & snapshot_path);
void uploadSnapshotImpl(const SnapshotFileInfo & snapshot_file_info);
/// Thread upload snapshots to S3 in the background
void snapshotS3Thread();
@ -60,7 +63,7 @@ public:
KeeperSnapshotManagerS3() = default;
void updateS3Configuration(const Poco::Util::AbstractConfiguration &, const MultiVersion<Macros>::Version &) {}
void uploadSnapshot(const std::string &, [[maybe_unused]] bool async_upload = true) {}
void uploadSnapshot(const SnapshotFileInfo &, [[maybe_unused]] bool async_upload = true) {}
void startup(const Poco::Util::AbstractConfiguration &, const MultiVersion<Macros>::Version &) {}

View File

@ -14,6 +14,8 @@
#include <Common/logger_useful.h>
#include "Coordination/KeeperStorage.h"
#include <Disks/DiskLocal.h>
namespace ProfileEvents
{
@ -33,17 +35,11 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int SYSTEM_ERROR;
}
namespace
{
}
KeeperStateMachine::KeeperStateMachine(
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_,
const std::string & snapshots_path_,
const CoordinationSettingsPtr & coordination_settings_,
const KeeperContextPtr & keeper_context_,
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
@ -52,7 +48,6 @@ KeeperStateMachine::KeeperStateMachine(
: commit_callback(commit_callback_)
, coordination_settings(coordination_settings_)
, snapshot_manager(
snapshots_path_,
coordination_settings->snapshots_to_keep,
keeper_context_,
coordination_settings->compress_snapshots_with_zstd_format,
@ -69,6 +64,16 @@ KeeperStateMachine::KeeperStateMachine(
{
}
namespace
{
bool isLocalDisk(const IDisk & disk)
{
return dynamic_cast<const DiskLocal *>(&disk) != nullptr;
}
}
void KeeperStateMachine::init()
{
/// Do everything without mutexes, no other threads exist.
@ -83,9 +88,13 @@ void KeeperStateMachine::init()
try
{
auto snapshot_deserialization_result
= snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index));
latest_snapshot_path = snapshot_manager.getLatestSnapshotPath();
latest_snapshot_buf = snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index);
auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_buf);
latest_snapshot_info = snapshot_manager.getLatestSnapshotInfo();
if (isLocalDisk(*latest_snapshot_info.disk))
latest_snapshot_buf = nullptr;
storage = std::move(snapshot_deserialization_result.storage);
latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta;
cluster_config = snapshot_deserialization_result.cluster_config;
@ -276,7 +285,7 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
std::abort();
}
if (keeper_context->digest_enabled && request_for_session.digest)
if (keeper_context->digestEnabled() && request_for_session.digest)
assertDigest(*request_for_session.digest, storage->getNodesDigest(false), *request_for_session.request, false);
return true;
@ -333,7 +342,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
response_for_session.session_id);
}
if (keeper_context->digest_enabled && request_for_session->digest)
if (keeper_context->digestEnabled() && request_for_session->digest)
assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true);
}
@ -371,8 +380,13 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
{ /// deserialize and apply snapshot to storage
std::lock_guard lock(storage_and_responses_lock);
auto snapshot_deserialization_result
= snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx()));
SnapshotDeserializationResult snapshot_deserialization_result;
if (latest_snapshot_ptr)
snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr);
else
snapshot_deserialization_result
= snapshot_manager.deserializeSnapshotFromBuffer(snapshot_manager.deserializeSnapshotBufferFromDisk(s.get_last_log_idx()));
/// maybe some logs were preprocessed with log idx larger than the snapshot idx
/// we have to apply them to the new storage
@ -464,19 +478,24 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
}
else
{
auto [path, error_code] = snapshot_manager.serializeSnapshotToDisk(*snapshot);
if (error_code)
{
throw Exception(
ErrorCodes::SYSTEM_ERROR,
"Snapshot {} was created failed, error: {}",
snapshot->snapshot_meta->get_last_log_idx(),
error_code.message());
}
latest_snapshot_path = path;
latest_snapshot_meta = snapshot->snapshot_meta;
/// we rely on the fact that the snapshot disk cannot be changed during runtime
if (isLocalDisk(*keeper_context->getLatestSnapshotDisk()))
{
auto snapshot_info = snapshot_manager.serializeSnapshotToDisk(*snapshot);
latest_snapshot_info = std::move(snapshot_info);
latest_snapshot_buf = nullptr;
}
else
{
auto snapshot_buf = snapshot_manager.serializeSnapshotToBuffer(*snapshot);
auto snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(*snapshot_buf, snapshot->snapshot_meta->get_last_log_idx());
latest_snapshot_info = std::move(snapshot_info);
latest_snapshot_buf = std::move(snapshot_buf);
}
ProfileEvents::increment(ProfileEvents::KeeperSnapshotCreations);
LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), path);
LOG_DEBUG(log, "Created persistent snapshot {} with path {}", latest_snapshot_meta->get_last_log_idx(), latest_snapshot_info.path);
}
}
@ -500,19 +519,19 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res
when_done(ret, exception);
return ret ? latest_snapshot_path : "";
return ret ? latest_snapshot_info : SnapshotFileInfo{};
};
if (keeper_context->server_state == KeeperContext::Phase::SHUTDOWN)
if (keeper_context->getServerState() == KeeperContext::Phase::SHUTDOWN)
{
LOG_INFO(log, "Creating a snapshot during shutdown because 'create_snapshot_on_exit' is enabled.");
auto snapshot_path = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot));
auto snapshot_file_info = snapshot_task.create_snapshot(std::move(snapshot_task.snapshot));
if (!snapshot_path.empty() && snapshot_manager_s3)
if (!snapshot_file_info.path.empty() && snapshot_manager_s3)
{
LOG_INFO(log, "Uploading snapshot {} during shutdown because 'upload_snapshot_on_exit' is enabled.", snapshot_path);
snapshot_manager_s3->uploadSnapshot(snapshot_path, /* asnyc_upload */ false);
LOG_INFO(log, "Uploading snapshot {} during shutdown because 'upload_snapshot_on_exit' is enabled.", snapshot_file_info.path);
snapshot_manager_s3->uploadSnapshot(snapshot_file_info, /* asnyc_upload */ false);
}
return;
@ -533,14 +552,20 @@ void KeeperStateMachine::save_logical_snp_obj(
nuraft::ptr<nuraft::buffer> snp_buf = s.serialize();
nuraft::ptr<nuraft::snapshot> cloned_meta = nuraft::snapshot::deserialize(*snp_buf);
nuraft::ptr<nuraft::buffer> cloned_buffer;
/// we rely on the fact that the snapshot disk cannot be changed during runtime
if (!isLocalDisk(*keeper_context->getSnapshotDisk()))
cloned_buffer = nuraft::buffer::clone(data);
try
{
std::lock_guard lock(snapshots_lock);
/// Serialize snapshot to disk
auto result_path = snapshot_manager.serializeSnapshotBufferToDisk(data, s.get_last_log_idx());
latest_snapshot_path = result_path;
latest_snapshot_info = snapshot_manager.serializeSnapshotBufferToDisk(data, s.get_last_log_idx());
latest_snapshot_meta = cloned_meta;
LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), result_path);
latest_snapshot_buf = std::move(cloned_buffer);
LOG_DEBUG(log, "Saved snapshot {} to path {}", s.get_last_log_idx(), latest_snapshot_info.path);
obj_id++;
ProfileEvents::increment(ProfileEvents::KeeperSaveSnapshot);
}
@ -600,11 +625,23 @@ int KeeperStateMachine::read_logical_snp_obj(
latest_snapshot_meta->get_last_log_idx());
return -1;
}
if (bufferFromFile(log, latest_snapshot_path, data_out))
const auto & [path, disk] = latest_snapshot_info;
if (isLocalDisk(*disk))
{
LOG_WARNING(log, "Error reading snapshot {} from {}", s.get_last_log_idx(), latest_snapshot_path);
return -1;
auto full_path = fs::path(disk->getPath()) / path;
if (bufferFromFile(log, full_path, data_out))
{
LOG_WARNING(log, "Error reading snapshot {} from {}", s.get_last_log_idx(), full_path);
return -1;
}
}
else
{
chassert(latest_snapshot_buf);
data_out = nuraft::buffer::clone(*latest_snapshot_buf);
}
is_last_obj = true;
ProfileEvents::increment(ProfileEvents::KeeperReadSnapshot);

View File

@ -26,7 +26,6 @@ public:
KeeperStateMachine(
ResponsesQueue & responses_queue_,
SnapshotsQueue & snapshots_queue_,
const std::string & snapshots_path_,
const CoordinationSettingsPtr & coordination_settings_,
const KeeperContextPtr & keeper_context_,
KeeperSnapshotManagerS3 * snapshot_manager_s3_,
@ -128,7 +127,7 @@ private:
/// In our state machine we always have a single snapshot which is stored
/// in memory in compressed (serialized) format.
SnapshotMetadataPtr latest_snapshot_meta = nullptr;
std::string latest_snapshot_path;
SnapshotFileInfo latest_snapshot_info;
nuraft::ptr<nuraft::buffer> latest_snapshot_buf = nullptr;
CoordinationSettingsPtr coordination_settings;

View File

@ -8,6 +8,7 @@
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromFile.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Disks/DiskLocal.h>
#include <Common/logger_useful.h>
namespace DB
@ -22,6 +23,8 @@ namespace ErrorCodes
namespace
{
const std::string copy_lock_file = "STATE_COPY_LOCK";
bool isLocalhost(const std::string & hostname)
{
try
@ -212,12 +215,14 @@ KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfigur
return result;
}
KeeperStateManager::KeeperStateManager(
int server_id_, const std::string & host, int port, const std::string & logs_path, const std::string & state_file_path)
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, KeeperContextPtr keeper_context_)
: my_server_id(server_id_)
, secure(false)
, log_store(nuraft::cs_new<KeeperLogStore>(logs_path, LogFileSettings{.force_sync =false, .compress_logs = false, .rotate_interval = 5000}))
, server_state_path(state_file_path)
, log_store(nuraft::cs_new<KeeperLogStore>(
LogFileSettings{.force_sync = false, .compress_logs = false, .rotate_interval = 5000},
keeper_context_))
, server_state_file_name("state")
, keeper_context(keeper_context_)
, logger(&Poco::Logger::get("KeeperStateManager"))
{
auto peer_config = nuraft::cs_new<nuraft::srv_config>(my_server_id, host + ":" + std::to_string(port));
@ -230,16 +235,15 @@ KeeperStateManager::KeeperStateManager(
KeeperStateManager::KeeperStateManager(
int my_server_id_,
const std::string & config_prefix_,
const std::string & log_storage_path,
const std::string & state_file_path,
const std::string & server_state_file_name_,
const Poco::Util::AbstractConfiguration & config,
const CoordinationSettingsPtr & coordination_settings)
const CoordinationSettingsPtr & coordination_settings,
KeeperContextPtr keeper_context_)
: my_server_id(my_server_id_)
, secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false))
, config_prefix(config_prefix_)
, configuration_wrapper(parseServersConfiguration(config, false))
, log_store(nuraft::cs_new<KeeperLogStore>(
log_storage_path,
LogFileSettings
{
.force_sync = coordination_settings->force_sync,
@ -247,8 +251,10 @@ KeeperStateManager::KeeperStateManager(
.rotate_interval = coordination_settings->rotate_log_storage_interval,
.max_size = coordination_settings->max_log_file_size,
.overallocate_size = coordination_settings->log_file_overallocate_size
}))
, server_state_path(state_file_path)
},
keeper_context_))
, server_state_file_name(server_state_file_name_)
, keeper_context(keeper_context_)
, logger(&Poco::Logger::get("KeeperStateManager"))
{
}
@ -287,16 +293,21 @@ void KeeperStateManager::save_config(const nuraft::cluster_config & config)
configuration_wrapper.cluster_config = nuraft::cluster_config::deserialize(*buf);
}
const std::filesystem::path & KeeperStateManager::getOldServerStatePath()
const String & KeeperStateManager::getOldServerStatePath()
{
static auto old_path = [this]
{
return server_state_path.parent_path() / (server_state_path.filename().generic_string() + "-OLD");
return server_state_file_name + "-OLD";
}();
return old_path;
}
DiskPtr KeeperStateManager::getStateFileDisk() const
{
return keeper_context->getStateFileDisk();
}
namespace
{
enum ServerStateVersion : uint8_t
@ -312,51 +323,61 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state)
{
const auto & old_path = getOldServerStatePath();
if (std::filesystem::exists(server_state_path))
std::filesystem::rename(server_state_path, old_path);
auto disk = getStateFileDisk();
WriteBufferFromFile server_state_file(server_state_path, DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
if (disk->exists(server_state_file_name))
{
auto buf = disk->writeFile(copy_lock_file);
buf->finalize();
disk->copyFile(server_state_file_name, *disk, old_path);
disk->removeFile(copy_lock_file);
disk->removeFile(old_path);
}
auto server_state_file = disk->writeFile(server_state_file_name);
auto buf = state.serialize();
// calculate checksum
SipHash hash;
hash.update(current_server_state_version);
hash.update(reinterpret_cast<const char *>(buf->data_begin()), buf->size());
writeIntBinary(hash.get64(), server_state_file);
writeIntBinary(hash.get64(), *server_state_file);
writeIntBinary(static_cast<uint8_t>(current_server_state_version), server_state_file);
writeIntBinary(static_cast<uint8_t>(current_server_state_version), *server_state_file);
server_state_file.write(reinterpret_cast<const char *>(buf->data_begin()), buf->size());
server_state_file.sync();
server_state_file.close();
server_state_file->write(reinterpret_cast<const char *>(buf->data_begin()), buf->size());
server_state_file->sync();
server_state_file->finalize();
std::filesystem::remove(old_path);
disk->removeFileIfExists(old_path);
}
nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
{
const auto & old_path = getOldServerStatePath();
const auto try_read_file = [this](const auto & path) -> nuraft::ptr<nuraft::srv_state>
auto disk = getStateFileDisk();
const auto try_read_file = [&](const auto & path) -> nuraft::ptr<nuraft::srv_state>
{
try
{
ReadBufferFromFile read_buf(path);
auto content_size = read_buf.getFileSize();
auto read_buf = disk->readFile(path);
auto content_size = read_buf->getFileSize();
if (content_size == 0)
return nullptr;
uint64_t read_checksum{0};
readIntBinary(read_checksum, read_buf);
readIntBinary(read_checksum, *read_buf);
uint8_t version;
readIntBinary(version, read_buf);
readIntBinary(version, *read_buf);
auto buffer_size = content_size - sizeof read_checksum - sizeof version;
auto state_buf = nuraft::buffer::alloc(buffer_size);
read_buf.readStrict(reinterpret_cast<char *>(state_buf->data_begin()), buffer_size);
read_buf->readStrict(reinterpret_cast<char *>(state_buf->data_begin()), buffer_size);
SipHash hash;
hash.update(version);
@ -366,15 +387,15 @@ nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
{
constexpr auto error_format = "Invalid checksum while reading state from {}. Got {}, expected {}";
#ifdef NDEBUG
LOG_ERROR(logger, error_format, path.generic_string(), hash.get64(), read_checksum);
LOG_ERROR(logger, error_format, path, hash.get64(), read_checksum);
return nullptr;
#else
throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, path.generic_string(), hash.get64(), read_checksum);
throw Exception(ErrorCodes::CORRUPTED_DATA, error_format, disk->getPath() + path, hash.get64(), read_checksum);
#endif
}
auto state = nuraft::srv_state::deserialize(*state_buf);
LOG_INFO(logger, "Read state from {}", path.generic_string());
LOG_INFO(logger, "Read state from {}", fs::path(disk->getPath()) / path);
return state;
}
catch (const std::exception & e)
@ -385,37 +406,45 @@ nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
throw;
}
LOG_ERROR(logger, "Failed to deserialize state from {}", path.generic_string());
LOG_ERROR(logger, "Failed to deserialize state from {}", disk->getPath() + path);
return nullptr;
}
};
if (std::filesystem::exists(server_state_path))
if (disk->exists(server_state_file_name))
{
auto state = try_read_file(server_state_path);
auto state = try_read_file(server_state_file_name);
if (state)
{
if (std::filesystem::exists(old_path))
std::filesystem::remove(old_path);
disk->removeFileIfExists(old_path);
return state;
}
std::filesystem::remove(server_state_path);
disk->removeFile(server_state_file_name);
}
if (std::filesystem::exists(old_path))
if (disk->exists(old_path))
{
auto state = try_read_file(old_path);
if (state)
if (disk->exists(copy_lock_file))
{
std::filesystem::rename(old_path, server_state_path);
return state;
disk->removeFile(old_path);
disk->removeFile(copy_lock_file);
}
std::filesystem::remove(old_path);
else
{
auto state = try_read_file(old_path);
if (state)
{
disk->moveFile(old_path, server_state_file_name);
return state;
}
disk->removeFile(old_path);
}
}
else if (disk->exists(copy_lock_file))
{
disk->removeFile(copy_lock_file);
}
LOG_WARNING(logger, "No state was read");

View File

@ -39,18 +39,17 @@ public:
KeeperStateManager(
int server_id_,
const std::string & config_prefix_,
const std::string & log_storage_path,
const std::string & state_file_path,
const std::string & server_state_file_name_,
const Poco::Util::AbstractConfiguration & config,
const CoordinationSettingsPtr & coordination_settings);
const CoordinationSettingsPtr & coordination_settings,
KeeperContextPtr keeper_context_);
/// Constructor for tests
KeeperStateManager(
int server_id_,
const std::string & host,
int port,
const std::string & logs_path,
const std::string & state_file_path);
KeeperContextPtr keeper_context_);
void loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep);
@ -111,7 +110,9 @@ public:
ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const;
private:
const std::filesystem::path & getOldServerStatePath();
const String & getOldServerStatePath();
DiskPtr getStateFileDisk() const;
/// Wrapper struct for Keeper cluster config. We parse this
/// info from XML files.
@ -136,7 +137,9 @@ private:
nuraft::ptr<KeeperLogStore> log_store;
const std::filesystem::path server_state_path;
const String server_state_file_name;
KeeperContextPtr keeper_context;
Poco::Logger * logger;

View File

@ -128,7 +128,7 @@ KeeperStorage::ResponsesForSessions processWatchesImpl(
watches.erase(watch_it);
}
auto parent_path = parentPath(path);
auto parent_path = parentNodePath(path);
Strings paths_to_check_for_list_watches;
if (event_type == Coordination::Event::CREATED)
@ -276,23 +276,23 @@ void KeeperStorage::initializeSystemNodes()
[](auto & node)
{
++node.stat.numChildren;
node.addChild(getBaseName(keeper_system_path));
node.addChild(getBaseNodeName(keeper_system_path));
}
);
addDigest(updated_root_it->value, "/");
}
// insert child system nodes
for (const auto & [path, data] : keeper_context->system_nodes_with_data)
for (const auto & [path, data] : keeper_context->getSystemNodesWithData())
{
assert(path.starts_with(keeper_system_path));
Node child_system_node;
child_system_node.setData(data);
auto [map_key, _] = container.insert(std::string{path}, child_system_node);
/// Take child path from key owned by map.
auto child_path = getBaseName(map_key->getKey());
auto child_path = getBaseNodeName(map_key->getKey());
container.updateValue(
parentPath(StringRef(path)),
parentNodePath(StringRef(path)),
[child_path](auto & parent)
{
// don't update stats so digest is okay
@ -728,7 +728,7 @@ bool KeeperStorage::createNode(
bool is_sequental,
Coordination::ACLs node_acls)
{
auto parent_path = parentPath(path);
auto parent_path = parentNodePath(path);
auto node_it = container.find(parent_path);
if (node_it == container.end())
@ -751,7 +751,7 @@ bool KeeperStorage::createNode(
created_node.is_sequental = is_sequental;
auto [map_key, _] = container.insert(path, created_node);
/// Take child path from key owned by map.
auto child_path = getBaseName(map_key->getKey());
auto child_path = getBaseNodeName(map_key->getKey());
container.updateValue(
parent_path,
[child_path](KeeperStorage::Node & parent)
@ -781,8 +781,8 @@ bool KeeperStorage::removeNode(const std::string & path, int32_t version)
acl_map.removeUsage(prev_node.acl_id);
container.updateValue(
parentPath(path),
[child_basename = getBaseName(node_it->key)](KeeperStorage::Node & parent)
parentNodePath(path),
[child_basename = getBaseNodeName(node_it->key)](KeeperStorage::Node & parent)
{
parent.removeChild(child_basename);
chassert(parent.stat.numChildren == static_cast<int32_t>(parent.getChildren().size()));
@ -866,7 +866,7 @@ Coordination::ACLs getNodeACLs(KeeperStorage & storage, StringRef path, bool is_
void handleSystemNodeModification(const KeeperContext & keeper_context, std::string_view error_msg)
{
if (keeper_context.server_state == KeeperContext::Phase::INIT && !keeper_context.ignore_system_path_on_startup)
if (keeper_context.getServerState() == KeeperContext::Phase::INIT && !keeper_context.ignoreSystemPathOnStartup())
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"{}. Ignoring it can lead to data loss. "
@ -929,7 +929,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
auto path = zk_request->getPath();
return storage.checkACL(parentPath(path), Coordination::ACL::Create, session_id, is_local);
return storage.checkACL(parentNodePath(path), Coordination::ACL::Create, session_id, is_local);
}
std::vector<KeeperStorage::Delta>
@ -940,7 +940,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
std::vector<KeeperStorage::Delta> new_deltas;
auto parent_path = parentPath(request.path);
auto parent_path = parentNodePath(request.path);
auto parent_node = storage.uncommitted_state.getNode(parent_path);
if (parent_node == nullptr)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNONODE}};
@ -971,7 +971,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr
if (storage.uncommitted_state.getNode(path_created))
return {KeeperStorage::Delta{zxid, Coordination::Error::ZNODEEXISTS}};
if (getBaseName(path_created).size == 0)
if (getBaseNodeName(path_created).size == 0)
return {KeeperStorage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}};
Coordination::ACLs node_acls;
@ -1121,7 +1121,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
{
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
return storage.checkACL(parentPath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
return storage.checkACL(parentNodePath(zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local);
}
using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor;
@ -1143,7 +1143,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
const auto update_parent_pzxid = [&]()
{
auto parent_path = parentPath(request.path);
auto parent_path = parentNodePath(request.path);
if (!storage.uncommitted_state.getNode(parent_path))
return;
@ -1178,7 +1178,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr
update_parent_pzxid();
new_deltas.emplace_back(
std::string{parentPath(request.path)},
std::string{parentNodePath(request.path)},
zxid,
KeeperStorage::UpdateNodeDelta{[](KeeperStorage::Node & parent)
{
@ -1321,7 +1321,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce
request.version});
new_deltas.emplace_back(
parentPath(request.path).toString(),
parentNodePath(request.path).toString(),
zxid,
KeeperStorage::UpdateNodeDelta
{
@ -1481,7 +1481,7 @@ struct KeeperStorageCheckRequestProcessor final : public KeeperStorageRequestPro
bool checkAuth(KeeperStorage & storage, int64_t session_id, bool is_local) const override
{
auto path = zk_request->getPath();
return storage.checkACL(check_not_exists ? parentPath(path) : path, Coordination::ACL::Read, session_id, is_local);
return storage.checkACL(check_not_exists ? parentNodePath(path) : path, Coordination::ACL::Read, session_id, is_local);
}
std::vector<KeeperStorage::Delta>
@ -2024,7 +2024,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFactory()
UInt64 KeeperStorage::calculateNodesDigest(UInt64 current_digest, const std::vector<Delta> & new_deltas) const
{
if (!keeper_context->digest_enabled)
if (!keeper_context->digestEnabled())
return current_digest;
std::unordered_map<std::string_view, std::shared_ptr<Node>> updated_nodes;
@ -2122,7 +2122,7 @@ void KeeperStorage::preprocessRequest(
TransactionInfo transaction{.zxid = new_last_zxid};
uint64_t new_digest = getNodesDigest(false).value;
SCOPE_EXIT({
if (keeper_context->digest_enabled)
if (keeper_context->digestEnabled())
// if the version of digest we got from the leader is the same as the one this instances has, we can simply copy the value
// and just check the digest on the commit
// a mistake can happen while applying the changes to the uncommitted_state so for now let's just recalculate the digest here also
@ -2145,7 +2145,7 @@ void KeeperStorage::preprocessRequest(
{
new_deltas.emplace_back
(
parentPath(ephemeral_path).toString(),
parentNodePath(ephemeral_path).toString(),
new_last_zxid,
UpdateNodeDelta
{
@ -2338,7 +2338,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing)
KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const
{
if (!keeper_context->digest_enabled)
if (!keeper_context->digestEnabled())
return {.version = DigestVersion::NO_DIGEST};
if (committed || uncommitted_transactions.empty())
@ -2349,13 +2349,13 @@ KeeperStorage::Digest KeeperStorage::getNodesDigest(bool committed) const
void KeeperStorage::removeDigest(const Node & node, const std::string_view path)
{
if (keeper_context->digest_enabled)
if (keeper_context->digestEnabled())
nodes_digest -= node.getDigest(path);
}
void KeeperStorage::addDigest(const Node & node, const std::string_view path)
{
if (keeper_context->digest_enabled)
if (keeper_context->digestEnabled())
{
node.invalidateDigestCache();
nodes_digest += node.getDigest(path);

View File

@ -0,0 +1,321 @@
#include <Interpreters/Context.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/Macros.h>
#include <Common/ThreadPool.h>
#include <Core/ServerSettings.h>
#include <boost/noncopyable.hpp>
#include <memory>
#include <cassert>
namespace ProfileEvents
{
extern const Event ContextLock;
}
namespace CurrentMetrics
{
extern const Metric ContextLockWait;
extern const Metric BackgroundSchedulePoolTask;
extern const Metric BackgroundSchedulePoolSize;
extern const Metric IOWriterThreads;
extern const Metric IOWriterThreadsActive;
}
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
struct ContextSharedPart : boost::noncopyable
{
ContextSharedPart()
: macros(std::make_unique<Macros>())
{}
/// For access of most of shared objects. Recursive mutex.
mutable std::recursive_mutex mutex;
mutable std::mutex keeper_dispatcher_mutex;
mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_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.
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 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;
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
};
Context::Context() = default;
Context::~Context() = default;
Context::Context(const Context &) = default;
Context & Context::operator=(const 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)
{
auto res = std::shared_ptr<Context>(new Context);
res->shared = shared;
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<std::recursive_mutex> Context::getLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
return std::unique_lock(shared->mutex);
}
String Context::getPath() const
{
auto lock = getLock();
return shared->path;
}
void Context::setPath(const String & path)
{
auto lock = getLock();
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
{
auto lock = getLock();
if (!shared->schedule_pool)
{
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
{
auto lock = getLock();
switch (type)
{
case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER:
{
if (!shared->asynchronous_remote_fs_reader)
shared->asynchronous_remote_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->asynchronous_remote_fs_reader;
}
case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->asynchronous_local_fs_reader)
shared->asynchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->asynchronous_local_fs_reader;
}
case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER:
{
if (!shared->synchronous_local_fs_reader)
shared->synchronous_local_fs_reader = createThreadPoolReader(type, getConfigRef());
return *shared->synchronous_local_fs_reader;
}
}
}
std::shared_ptr<FilesystemCacheLog> Context::getFilesystemCacheLog() const
{
return nullptr;
}
std::shared_ptr<FilesystemReadPrefetchesLog> Context::getFilesystemReadPrefetchesLog() const
{
return nullptr;
}
void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->config = config;
}
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
{
auto lock = getLock();
return shared->config ? *shared->config : Poco::Util::Application::instance().config();
}
std::shared_ptr<AsyncReadCounters> Context::getAsyncReadCounters() const
{
auto lock = getLock();
if (!async_read_counters)
async_read_counters = std::make_shared<AsyncReadCounters>();
return async_read_counters;
}
ThreadPool & Context::getThreadPoolWriter() const
{
const auto & config = getConfigRef();
auto lock = getLock();
if (!shared->threadpool_writer)
{
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, 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{};
}
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(getConfigRef(), getMacros());
}
}

View File

@ -0,0 +1,120 @@
#pragma once
#include <Interpreters/Context_fwd.h>
#include <Coordination/KeeperDispatcher.h>
#include <Common/MultiVersion.h>
#include <Common/RemoteHostFilter.h>
#include <Disks/IO/getThreadPoolReader.h>
#include <Core/Settings.h>
#include <Core/BackgroundSchedulePool.h>
#include <IO/AsyncReadCounters.h>
#include <Poco/Util/Application.h>
#include <memory>
namespace DB
{
struct ContextSharedPart;
class Macros;
class FilesystemCacheLog;
class FilesystemReadPrefetchesLog;
/// 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 Context : public std::enable_shared_from_this<Context>
{
private:
/// Use copy constructor or createGlobal() instead
Context();
Context(const Context &);
Context & operator=(const Context &);
std::unique_lock<std::recursive_mutex> getLock() const;
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:
/// Create initial Context with ContextShared and etc.
static ContextMutablePtr createGlobal(ContextSharedPart * shared);
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;
IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type) const;
std::shared_ptr<AsyncReadCounters> getAsyncReadCounters() const;
ThreadPool & getThreadPoolWriter() const;
ThrottlerPtr getRemoteReadThrottler() const;
ThrottlerPtr getRemoteWriteThrottler() const;
ThrottlerPtr getLocalReadThrottler() const;
ThrottlerPtr getLocalWriteThrottler() const;
ReadSettings getReadSettings() 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);
};
}

View File

@ -0,0 +1,24 @@
#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);
}
}

View File

@ -0,0 +1,14 @@
#include <Common/CurrentThread.h>
namespace DB
{
void CurrentThread::detachFromGroupIfNotDetached()
{
}
void CurrentThread::attachToGroup(const ThreadGroupPtr &)
{
}
}

View File

@ -1,87 +0,0 @@
#include <Coordination/TinyContext.h>
#include <Common/Exception.h>
#include <Coordination/KeeperDispatcher.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void TinyContext::setConfig(const ConfigurationPtr & config_)
{
std::lock_guard lock(keeper_dispatcher_mutex);
config = config_;
}
const Poco::Util::AbstractConfiguration & TinyContext::getConfigRef() const
{
std::lock_guard lock(keeper_dispatcher_mutex);
return config ? *config : Poco::Util::Application::instance().config();
}
void TinyContext::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const
{
const auto & config_ref = getConfigRef();
std::lock_guard lock(keeper_dispatcher_mutex);
if (keeper_dispatcher)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times");
if (config_ref.has("keeper_server"))
{
keeper_dispatcher = std::make_shared<KeeperDispatcher>();
MultiVersion<Macros>::Version macros;
if (config_ref.has("macros"))
macros = std::make_unique<Macros>(config_ref, "macros", &Poco::Logger::get("TinyContext"));
keeper_dispatcher->initialize(config_ref, true, start_async, macros);
}
}
std::shared_ptr<KeeperDispatcher> TinyContext::getKeeperDispatcher() const
{
std::lock_guard lock(keeper_dispatcher_mutex);
if (!keeper_dispatcher)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests");
return keeper_dispatcher;
}
std::shared_ptr<KeeperDispatcher> TinyContext::tryGetKeeperDispatcher() const
{
std::lock_guard lock(keeper_dispatcher_mutex);
return keeper_dispatcher;
}
void TinyContext::shutdownKeeperDispatcher() const
{
std::lock_guard lock(keeper_dispatcher_mutex);
if (keeper_dispatcher)
{
keeper_dispatcher->shutdown();
keeper_dispatcher.reset();
}
}
void TinyContext::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock(keeper_dispatcher_mutex);
if (!keeper_dispatcher)
return;
MultiVersion<Macros>::Version macros;
if (config_.has("macros"))
macros = std::make_unique<Macros>(config_, "macros", &Poco::Logger::get("TinyContext"));
keeper_dispatcher->updateConfiguration(config_, macros);
}
}

View File

@ -1,36 +0,0 @@
#pragma once
#include <memory>
#include <mutex>
#include <Poco/Util/Application.h>
#include <base/defines.h>
namespace DB
{
class KeeperDispatcher;
class TinyContext : public std::enable_shared_from_this<TinyContext>
{
public:
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);
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
void setConfig(const ConfigurationPtr & config);
const Poco::Util::AbstractConfiguration & getConfigRef() const;
private:
mutable std::mutex keeper_dispatcher_mutex;
mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher TSA_GUARDED_BY(keeper_dispatcher_mutex);
ConfigurationPtr config TSA_GUARDED_BY(keeper_dispatcher_mutex);
};
using TinyContextPtr = std::shared_ptr<TinyContext>;
}

View File

@ -139,8 +139,8 @@ int64_t deserializeStorageData(KeeperStorage & storage, ReadBuffer & in, Poco::L
{
if (itr.key != "/")
{
auto parent_path = parentPath(itr.key);
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseName(my_path)); ++value.stat.numChildren; });
auto parent_path = parentNodePath(itr.key);
storage.container.updateValue(parent_path, [my_path = itr.key] (KeeperStorage::Node & value) { value.addChild(getBaseNodeName(my_path)); ++value.stat.numChildren; });
}
}

View File

@ -21,7 +21,7 @@ static size_t findLastSlash(StringRef path)
return std::string::npos;
}
StringRef parentPath(StringRef path)
StringRef parentNodePath(StringRef path)
{
auto rslash_pos = findLastSlash(path);
if (rslash_pos > 0)
@ -29,7 +29,7 @@ StringRef parentPath(StringRef path)
return "/";
}
StringRef getBaseName(StringRef path)
StringRef getBaseNodeName(StringRef path)
{
size_t basename_start = findLastSlash(path);
return StringRef{path.data + basename_start + 1, path.size - basename_start - 1};

View File

@ -6,8 +6,8 @@
namespace DB
{
StringRef parentPath(StringRef path);
StringRef parentNodePath(StringRef path);
StringRef getBaseName(StringRef path);
StringRef getBaseNodeName(StringRef path);
}

File diff suppressed because it is too large Load Diff

View File

@ -378,6 +378,40 @@ 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

View File

@ -245,6 +245,12 @@ struct SettingFieldString
void readBinary(ReadBuffer & in);
};
#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
#define NORETURN [[noreturn]]
#else
#define NORETURN
#endif
struct SettingFieldMap
{
public:
@ -261,13 +267,15 @@ public:
operator const Map &() const { return value; } /// NOLINT
explicit operator Field() const { return value; }
String toString() const;
void parseFromString(const String & str);
NORETURN String toString() const;
NORETURN void parseFromString(const String & str);
void writeBinary(WriteBuffer & out) const;
void readBinary(ReadBuffer & in);
NORETURN void writeBinary(WriteBuffer & out) const;
NORETURN void readBinary(ReadBuffer & in);
};
#undef NORETURN
struct SettingFieldChar
{
public:

View File

@ -56,6 +56,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
size_t current_read_until_position = read_until_position ? read_until_position : object.bytes_size;
auto current_read_buffer_creator = [=, this]() { return read_buffer_creator(object_path, current_read_until_position); };
#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
if (with_cache)
{
auto cache_key = settings.remote_fs_cache->createKeyForPath(object_path);
@ -72,6 +73,7 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(c
read_until_position ? std::optional<size_t>(read_until_position) : std::nullopt,
cache_log);
}
#endif
return current_read_buffer_creator();
}

View File

@ -7,9 +7,7 @@
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
#include <Disks/IO/ThreadPoolReader.h>
#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
#include <Interpreters/Context.h>
#endif
namespace DB
{
@ -21,32 +19,10 @@ namespace ErrorCodes
IAsynchronousReader & getThreadPoolReader(FilesystemReaderType type)
{
#ifdef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
const auto & config = Poco::Util::Application::instance().config();
switch (type)
{
case FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER:
{
static auto asynchronous_remote_fs_reader = createThreadPoolReader(type, config);
return *asynchronous_remote_fs_reader;
}
case FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER:
{
static auto asynchronous_local_fs_reader = createThreadPoolReader(type, config);
return *asynchronous_local_fs_reader;
}
case FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER:
{
static auto synchronous_local_fs_reader = createThreadPoolReader(type, config);
return *synchronous_local_fs_reader;
}
}
#else
auto context = Context::getGlobalContextInstance();
if (!context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context not initialized");
return context->getThreadPoolReader(type);
#endif
}
std::unique_ptr<IAsynchronousReader> createThreadPoolReader(

View File

@ -11,7 +11,6 @@
#include <Common/logger_useful.h>
#include <Common/filesystemHelpers.h>
#include <Common/CurrentMetrics.h>
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.h>
#include <Disks/ObjectStorages/DiskObjectStorageTransaction.h>
#include <Disks/FakeDiskTransaction.h>
@ -530,24 +529,6 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage()
threadpool_size);
}
void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name)
{
object_storage = std::make_shared<CachedObjectStorage>(object_storage, cache, cache_settings, layer_name);
}
NameSet DiskObjectStorage::getCacheLayersNames() const
{
NameSet cache_layers;
auto current_object_storage = object_storage;
while (current_object_storage->supportsCache())
{
auto * cached_object_storage = assert_cast<CachedObjectStorage *>(current_object_storage.get());
cache_layers.insert(cached_object_storage->getCacheConfigName());
current_object_storage = cached_object_storage->getWrappedObjectStorage();
}
return cache_layers;
}
std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
const String & path,
const ReadSettings & settings,

View File

@ -181,20 +181,22 @@ public:
/// MergeTree table on this disk.
bool isWriteOnce() const override;
/// Add a cache layer.
/// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
/// There can be any number of cache layers:
/// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...))
void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name);
/// Get structure of object storage this disk works with. Examples:
/// DiskObjectStorage(S3ObjectStorage)
/// DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
/// DiskObjectStorage(CachedObjectStorage(CachedObjectStorage(S3ObjectStorage)))
String getStructure() const { return fmt::format("DiskObjectStorage-{}({})", getName(), object_storage->getName()); }
#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
/// Add a cache layer.
/// Example: DiskObjectStorage(S3ObjectStorage) -> DiskObjectStorage(CachedObjectStorage(S3ObjectStorage))
/// There can be any number of cache layers:
/// DiskObjectStorage(CachedObjectStorage(...CacheObjectStorage(S3ObjectStorage)...))
void wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name);
/// Get names of all cache layers. Name is how cache is defined in configuration file.
NameSet getCacheLayersNames() const override;
#endif
static std::shared_ptr<Executor> getAsyncExecutor(const std::string & log_name, size_t size);

View File

@ -0,0 +1,28 @@
#include <Disks/ObjectStorages/Cached/CachedObjectStorage.h>
#include <Disks/ObjectStorages/DiskObjectStorage.h>
#include <Common/assert_cast.h>
namespace DB
{
void DiskObjectStorage::wrapWithCache(FileCachePtr cache, const FileCacheSettings & cache_settings, const String & layer_name)
{
object_storage = std::make_shared<CachedObjectStorage>(object_storage, cache, cache_settings, layer_name);
}
NameSet DiskObjectStorage::getCacheLayersNames() const
{
NameSet cache_layers;
auto current_object_storage = object_storage;
while (current_object_storage->supportsCache())
{
auto * cached_object_storage = assert_cast<CachedObjectStorage *>(current_object_storage.get());
cache_layers.insert(cached_object_storage->getCacheConfigName());
current_object_storage = cached_object_storage->getWrappedObjectStorage();
}
return cache_layers;
}
}

View File

@ -63,7 +63,7 @@ public:
uint32_t getHardlinkCount(const std::string & /* path */) const override
{
return 1;
return 0;
}
bool supportsChmod() const override { return false; }

View File

@ -32,6 +32,8 @@ void registerDiskCache(DiskFactory & factory, bool global_skip_access_check);
void registerDiskLocalObjectStorage(DiskFactory & factory, bool global_skip_access_check);
#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
void registerDisks(bool global_skip_access_check)
{
auto & factory = DiskFactory::instance();
@ -61,4 +63,19 @@ void registerDisks(bool global_skip_access_check)
registerDiskLocalObjectStorage(factory, global_skip_access_check);
}
#else
void registerDisks(bool global_skip_access_check)
{
auto & factory = DiskFactory::instance();
registerDiskLocal(factory, global_skip_access_check);
#if USE_AWS_S3
registerDiskS3(factory, global_skip_access_check);
#endif
}
#endif
}

View File

@ -1,5 +1,6 @@
#include <IO/ZstdDeflatingAppendableWriteBuffer.h>
#include <Common/Exception.h>
#include "IO/ReadBufferFromFileBase.h"
#include <IO/ReadBufferFromFile.h>
namespace DB
@ -11,14 +12,16 @@ namespace ErrorCodes
}
ZstdDeflatingAppendableWriteBuffer::ZstdDeflatingAppendableWriteBuffer(
std::unique_ptr<WriteBufferFromFile> out_,
std::unique_ptr<WriteBufferFromFileBase> out_,
int compression_level,
bool append_to_existing_file_,
std::function<std::unique_ptr<ReadBufferFromFileBase>()> read_buffer_creator_,
size_t buf_size,
char * existing_memory,
size_t alignment)
: BufferWithOwnMemory(buf_size, existing_memory, alignment)
, out(std::move(out_))
, read_buffer_creator(std::move(read_buffer_creator_))
, append_to_existing_file(append_to_existing_file_)
{
cctx = ZSTD_createCCtx();
@ -194,13 +197,13 @@ void ZstdDeflatingAppendableWriteBuffer::addEmptyBlock()
bool ZstdDeflatingAppendableWriteBuffer::isNeedToAddEmptyBlock()
{
ReadBufferFromFile reader(out->getFileName());
auto fsize = reader.getFileSize();
auto reader = read_buffer_creator();
auto fsize = reader->getFileSize();
if (fsize > 3)
{
std::array<char, 3> result;
reader.seek(fsize - 3, SEEK_SET);
reader.readStrict(result.data(), 3);
reader->seek(fsize - 3, SEEK_SET);
reader->readStrict(result.data(), 3);
/// If we don't have correct block in the end, then we need to add it manually.
/// NOTE: maybe we can have the same bytes in case of data corruption/unfinished write.

View File

@ -5,6 +5,7 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteBufferDecorator.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFileBase.h>
#include <zstd.h>
@ -29,9 +30,10 @@ public:
static inline constexpr ZSTDLastBlock ZSTD_CORRECT_TERMINATION_LAST_BLOCK = {0x01, 0x00, 0x00};
ZstdDeflatingAppendableWriteBuffer(
std::unique_ptr<WriteBufferFromFile> out_,
std::unique_ptr<WriteBufferFromFileBase> out_,
int compression_level,
bool append_to_existing_file_,
std::function<std::unique_ptr<ReadBufferFromFileBase>()> read_buffer_creator_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr,
size_t alignment = 0);
@ -68,7 +70,8 @@ private:
/// Adding zstd empty block (ZSTD_CORRECT_TERMINATION_LAST_BLOCK) to out.working_buffer
void addEmptyBlock();
std::unique_ptr<WriteBufferFromFile> out;
std::unique_ptr<WriteBufferFromFileBase> out;
std::function<std::unique_ptr<ReadBufferFromFileBase>()> read_buffer_creator;
bool append_to_existing_file = false;
ZSTD_CCtx * cctx;

View File

@ -1,5 +1,7 @@
#pragma once
#ifndef CLICKHOUSE_PROGRAM_STANDALONE_BUILD
#include <base/types.h>
#include <Common/isLocalAddress.h>
#include <Common/MultiVersion.h>
@ -1240,3 +1242,9 @@ struct HTTPContext : public IHTTPContext
};
}
#else
#include <Coordination/Standalone/Context.h>
#endif

View File

@ -14,6 +14,9 @@ namespace QueryPlanOptimizations
void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes);
/// Second pass is used to apply read-in-order and attach a predicate to PK.
void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes);
/// Third pass is used to apply filters such as key conditions and skip indexes to the storages that support them.
/// After that it add CreateSetsStep for the subqueries that has not be used in the filters.
void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes);
/// Optimization (first pass) is a function applied to QueryPlan::Node.
/// It can read and update subtree of specified node.

View File

@ -28,8 +28,6 @@ void optimizePrimaryKeyCondition(const Stack & stack)
else
break;
}
source_step_with_filter->onAddFilterFinish();
}
}

View File

@ -167,7 +167,6 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
optimizePrewhere(stack, nodes);
optimizePrimaryKeyCondition(stack);
enableMemoryBoundMerging(*stack.back().node, nodes);
addPlansForSets(*stack.back().node, nodes);
stack.pop_back();
}
@ -178,5 +177,35 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
"No projection is used when optimize_use_projections = 1 and force_optimize_projection = 1");
}
void optimizeTreeThirdPass(QueryPlan::Node & root, QueryPlan::Nodes & nodes)
{
Stack stack;
stack.push_back({.node = &root});
while (!stack.empty())
{
/// NOTE: frame cannot be safely used after stack was modified.
auto & frame = stack.back();
/// Traverse all children first.
if (frame.next_child < frame.node->children.size())
{
auto next_frame = Frame{.node = frame.node->children[frame.next_child]};
++frame.next_child;
stack.push_back(next_frame);
continue;
}
if (auto * source_step_with_filter = dynamic_cast<SourceStepWithFilter *>(frame.node->step.get()))
{
source_step_with_filter->applyFilters();
}
addPlansForSets(*frame.node, nodes);
stack.pop_back();
}
}
}
}

View File

@ -482,6 +482,7 @@ void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_sett
QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes);
QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes);
QueryPlanOptimizations::optimizeTreeThirdPass(*root, nodes);
updateDataStreams(*root);
}

View File

@ -1299,13 +1299,10 @@ static void buildIndexes(
indexes->skip_indexes = std::move(skip_indexes);
}
void ReadFromMergeTree::onAddFilterFinish()
void ReadFromMergeTree::applyFilters()
{
if (!filter_nodes.nodes.empty())
{
auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info);
buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading);
}
auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info);
buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading);
}
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(

View File

@ -226,7 +226,7 @@ public:
size_t getNumStreams() const { return requested_num_streams; }
bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; }
void onAddFilterFinish() override;
void applyFilters() override;
private:
static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl(

View File

@ -37,7 +37,8 @@ public:
filter_dags.push_back(std::move(filter_dag));
}
virtual void onAddFilterFinish() {}
/// Apply filters that can optimize reading from storage.
virtual void applyFilters() {}
protected:
std::vector<ActionsDAGPtr> filter_dags;

View File

@ -415,41 +415,41 @@ void DataPartStorageOnDiskBase::backup(
MutableDataPartStoragePtr DataPartStorageOnDiskBase::freeze(
const std::string & to,
const std::string & dir_path,
bool make_source_readonly,
std::function<void(const DiskPtr &)> save_metadata_callback,
bool copy_instead_of_hardlink,
const NameSet & files_to_copy_instead_of_hardlinks,
DiskTransactionPtr external_transaction) const
const ClonePartParams & params) const
{
auto disk = volume->getDisk();
if (external_transaction)
external_transaction->createDirectories(to);
if (params.external_transaction)
params.external_transaction->createDirectories(to);
else
disk->createDirectories(to);
localBackup(disk, getRelativePath(), fs::path(to) / dir_path, make_source_readonly, {}, copy_instead_of_hardlink, files_to_copy_instead_of_hardlinks, external_transaction);
localBackup(disk, getRelativePath(), fs::path(to) / dir_path, params.make_source_readonly, {}, params.copy_instead_of_hardlink,
params.files_to_copy_instead_of_hardlinks, params.external_transaction);
if (save_metadata_callback)
save_metadata_callback(disk);
if (external_transaction)
if (params.external_transaction)
{
external_transaction->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt");
external_transaction->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt");
external_transaction->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME);
params.external_transaction->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt");
params.external_transaction->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt");
if (!params.keep_metadata_version)
params.external_transaction->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME);
}
else
{
disk->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt");
disk->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt");
disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME);
if (!params.keep_metadata_version)
disk->removeFileIfExists(fs::path(to) / dir_path / IMergeTreeDataPart::METADATA_VERSION_FILE_NAME);
}
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
/// Do not initialize storage in case of DETACH because part may be broken.
bool to_detached = dir_path.starts_with("detached/");
return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached && !external_transaction);
return create(single_disk_volume, to, dir_path, /*initialize=*/ !to_detached && !params.external_transaction);
}
MutableDataPartStoragePtr DataPartStorageOnDiskBase::clonePart(

View File

@ -62,11 +62,8 @@ public:
MutableDataPartStoragePtr freeze(
const std::string & to,
const std::string & dir_path,
bool make_source_readonly,
std::function<void(const DiskPtr &)> save_metadata_callback,
bool copy_instead_of_hardlink,
const NameSet & files_to_copy_instead_of_hardlinks,
DiskTransactionPtr external_transaction) const override;
const ClonePartParams & params) const override;
MutableDataPartStoragePtr clonePart(
const std::string & to,

View File

@ -63,6 +63,9 @@ using DiskPtr = std::shared_ptr<IDisk>;
class ISyncGuard;
using SyncGuardPtr = std::unique_ptr<ISyncGuard>;
class MergeTreeTransaction;
using MergeTreeTransactionPtr = std::shared_ptr<MergeTreeTransaction>;
class IBackupEntry;
using BackupEntryPtr = std::shared_ptr<const IBackupEntry>;
using BackupEntries = std::vector<std::pair<String, BackupEntryPtr>>;
@ -72,6 +75,17 @@ struct WriteSettings;
class TemporaryFileOnDisk;
struct HardlinkedFiles
{
/// Shared table uuid where hardlinks live
std::string source_table_shared_id;
/// Hardlinked from part
std::string source_part_name;
/// Hardlinked files list
NameSet hardlinks_from_source_part;
};
/// This is an abstraction of storage for data part files.
/// Ideally, it is assumed to contain read-only methods from IDisk.
/// It is not fulfilled now, but let's try our best.
@ -220,14 +234,24 @@ public:
/// If `external_transaction` is provided, the disk operations (creating directories, hardlinking,
/// etc) won't be applied immediately; instead, they'll be added to external_transaction, which the
/// caller then needs to commit.
struct ClonePartParams
{
MergeTreeTransactionPtr txn = NO_TRANSACTION_PTR;
HardlinkedFiles * hardlinked_files = nullptr;
bool copy_instead_of_hardlink = false;
NameSet files_to_copy_instead_of_hardlinks;
bool keep_metadata_version = false;
bool make_source_readonly = false;
DiskTransactionPtr external_transaction = nullptr;
std::optional<int32_t> metadata_version_to_write = std::nullopt;
};
virtual std::shared_ptr<IDataPartStorage> freeze(
const std::string & to,
const std::string & dir_path,
bool make_source_readonly,
std::function<void(const DiskPtr &)> save_metadata_callback,
bool copy_instead_of_hardlink,
const NameSet & files_to_copy_instead_of_hardlinks,
DiskTransactionPtr external_transaction = nullptr) const = 0;
const ClonePartParams & params) const = 0;
/// Make a full copy of a data part into 'to/dir_path' (possibly to a different disk).
virtual std::shared_ptr<IDataPartStorage> clonePart(

View File

@ -1362,6 +1362,10 @@ void IMergeTreeDataPart::loadColumns(bool require)
else
{
loaded_metadata_version = metadata_snapshot->getMetadataVersion();
old_part_with_no_metadata_version_on_disk = true;
if (storage.supportsReplication())
LOG_WARNING(storage.log, "Part {} doesn't have metadata version on disk, setting it to {}. "
"It's okay if the part was created by an old version of ClickHouse", name, loaded_metadata_version);
}
setColumns(loaded_columns, infos, loaded_metadata_version);
@ -1764,12 +1768,6 @@ void IMergeTreeDataPart::renameToDetached(const String & prefix)
DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const
{
auto storage_settings = storage.getSettings();
/// In case of zero-copy replication we copy directory instead of hardlinks
/// because hardlinks tracking doesn't work for detached parts.
bool copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication;
/// Avoid unneeded duplicates of broken parts if we try to detach the same broken part multiple times.
/// Otherwise it may pollute detached/ with dirs with _tryN suffix and we will fail to remove broken part after 10 attempts.
bool broken = !prefix.empty();
@ -1777,13 +1775,19 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix
if (!maybe_path_in_detached)
return nullptr;
/// In case of zero-copy replication we copy directory instead of hardlinks
/// because hardlinks tracking doesn't work for detached parts.
auto storage_settings = storage.getSettings();
IDataPartStorage::ClonePartParams params
{
.copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication,
.make_source_readonly = true
};
return getDataPartStorage().freeze(
storage.relative_data_path,
*maybe_path_in_detached,
/*make_source_readonly=*/ true,
/*save_metadata_callback=*/ {},
copy_instead_of_hardlink,
/*files_to_copy_instead_of_hardlinks=*/ {});
params);
}
MutableDataPartStoragePtr IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const

View File

@ -249,6 +249,9 @@ public:
/// Flag for keep S3 data when zero-copy replication over S3 turned on.
mutable bool force_keep_shared_data = false;
/// Some old parts don't have metadata version, so we set it to the current table's version when loading the part
bool old_part_with_no_metadata_version_on_disk = false;
using TTLInfo = MergeTreeDataPartTTLInfo;
using TTLInfos = MergeTreeDataPartTTLInfos;

View File

@ -4707,12 +4707,19 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExistsUnlocked(const MergeTre
return nullptr;
}
static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part)
static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part, ContextPtr local_context, int32_t metadata_version, bool sync)
{
/// Remove metadata version file and take it from table.
/// Currently we cannot attach parts with different schema, so
/// we can assume that it's equal to table's current schema.
part->removeMetadataVersion();
{
auto out_metadata = part->getDataPartStorage().writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, local_context->getWriteSettings());
writeText(metadata_version, *out_metadata);
out_metadata->finalize();
if (sync)
out_metadata->sync();
}
part->loadColumnsChecksumsIndexes(false, true);
part->modification_time = part->getDataPartStorage().getLastModified().epochTime();
@ -5844,7 +5851,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const
.withPartFormatFromDisk()
.build();
loadPartAndFixMetadataImpl(part);
loadPartAndFixMetadataImpl(part, local_context, getInMemoryMetadataPtr()->getMetadataVersion(), getSettings()->fsync_after_insert);
loaded_parts.push_back(part);
}
@ -7289,10 +7296,7 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info,
const StorageMetadataPtr & metadata_snapshot,
const MergeTreeTransactionPtr & txn,
HardlinkedFiles * hardlinked_files,
bool copy_instead_of_hardlink,
const NameSet & files_to_copy_instead_of_hardlinks)
const IDataPartStorage::ClonePartParams & params)
{
/// Check that the storage policy contains the disk where the src_part is located.
bool does_storage_policy_allow_same_disk = false;
@ -7343,16 +7347,24 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
}
String with_copy;
if (copy_instead_of_hardlink)
if (params.copy_instead_of_hardlink)
with_copy = " (copying data)";
auto dst_part_storage = src_part_storage->freeze(
relative_data_path,
tmp_dst_part_name,
/*make_source_readonly=*/ false,
/*save_metadata_callback=*/ {},
copy_instead_of_hardlink,
files_to_copy_instead_of_hardlinks);
params);
if (params.metadata_version_to_write.has_value())
{
chassert(!params.keep_metadata_version);
auto out_metadata = dst_part_storage->writeFile(IMergeTreeDataPart::METADATA_VERSION_FILE_NAME, 4096, getContext()->getWriteSettings());
writeText(metadata_snapshot->getMetadataVersion(), *out_metadata);
out_metadata->finalize();
if (getSettings()->fsync_after_insert)
out_metadata->sync();
}
LOG_DEBUG(log, "Clone{} part {} to {}{}",
src_flushed_tmp_part ? " flushed" : "",
@ -7364,18 +7376,18 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
.withPartFormatFromDisk()
.build();
if (!copy_instead_of_hardlink && hardlinked_files)
if (!params.copy_instead_of_hardlink && params.hardlinked_files)
{
hardlinked_files->source_part_name = src_part->name;
hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID();
params.hardlinked_files->source_part_name = src_part->name;
params.hardlinked_files->source_table_shared_id = src_part->storage.getTableSharedID();
for (auto it = src_part->getDataPartStorage().iterate(); it->isValid(); it->next())
{
if (!files_to_copy_instead_of_hardlinks.contains(it->name())
if (!params.files_to_copy_instead_of_hardlinks.contains(it->name())
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
{
hardlinked_files->hardlinks_from_source_part.insert(it->name());
params.hardlinked_files->hardlinks_from_source_part.insert(it->name());
}
}
@ -7386,18 +7398,18 @@ std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> MergeTreeData::cloneAn
for (auto it = projection_storage.iterate(); it->isValid(); it->next())
{
auto file_name_with_projection_prefix = fs::path(projection_storage.getPartDirectory()) / it->name();
if (!files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix)
if (!params.files_to_copy_instead_of_hardlinks.contains(file_name_with_projection_prefix)
&& it->name() != IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME_DEPRECATED
&& it->name() != IMergeTreeDataPart::TXN_VERSION_METADATA_FILE_NAME)
{
hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix);
params.hardlinked_files->hardlinks_from_source_part.insert(file_name_with_projection_prefix);
}
}
}
}
/// We should write version metadata on part creation to distinguish it from parts that were created without transaction.
TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID;
TransactionID tid = params.txn ? params.txn->tid : Tx::PrehistoricTID;
dst_data_part->version.setCreationTID(tid, nullptr);
dst_data_part->storeVersionMetadata();
@ -7579,13 +7591,15 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(
createAndStoreFreezeMetadata(disk, part, fs::path(backup_part_path) / part->getDataPartStorage().getPartDirectory());
};
IDataPartStorage::ClonePartParams params
{
.make_source_readonly = true
};
auto new_storage = data_part_storage->freeze(
backup_part_path,
part->getDataPartStorage().getPartDirectory(),
/*make_source_readonly=*/ true,
callback,
/*copy_instead_of_hardlink=*/ false,
/*files_to_copy_instead_of_hardlinks=*/ {});
params);
part->is_frozen.store(true, std::memory_order_relaxed);
result.push_back(PartitionCommandResultInfo{

View File

@ -828,21 +828,10 @@ public:
MergeTreeData & checkStructureAndGetMergeTreeData(const StoragePtr & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const;
struct HardlinkedFiles
{
/// Shared table uuid where hardlinks live
std::string source_table_shared_id;
/// Hardlinked from part
std::string source_part_name;
/// Hardlinked files list
NameSet hardlinks_from_source_part;
};
std::pair<MergeTreeData::MutableDataPartPtr, scope_guard> cloneAndLoadDataPartOnSameDisk(
const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix,
const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot,
const MergeTreeTransactionPtr & txn, HardlinkedFiles * hardlinked_files,
bool copy_instead_of_hardlink, const NameSet & files_to_copy_instead_of_hardlinks);
const IDataPartStorage::ClonePartParams & params);
virtual std::vector<MergeTreeMutationStatus> getMutationsStatus() const = 0;

View File

@ -110,6 +110,9 @@ static void collectColumns(const RPNBuilderTreeNode & node, const NameSet & colu
if (node.isConstant())
return;
if (node.isSubqueryOrSet())
return;
if (!node.isFunction())
{
auto column_name = node.getColumnName();

View File

@ -153,20 +153,22 @@ static void splitAndModifyMutationCommands(
/// But we don't know for sure what happened.
auto part_metadata_version = part->getMetadataVersion();
auto table_metadata_version = metadata_snapshot->getMetadataVersion();
/// StorageMergeTree does not have metadata version
if (table_metadata_version <= part_metadata_version && part->storage.supportsReplication())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent "
"in table {} with metadata version {}",
part->name, part_metadata_version, column.name,
part->storage.getStorageID().getNameForLogs(), table_metadata_version);
if (part_metadata_version < table_metadata_version)
bool allow_equal_versions = part_metadata_version == table_metadata_version && part->old_part_with_no_metadata_version_on_disk;
if (part_metadata_version < table_metadata_version || allow_equal_versions)
{
LOG_WARNING(log, "Ignoring column {} from part {} with metadata version {} because there is no such column "
"in table {} with metadata version {}. Assuming the column was dropped", column.name, part->name,
part_metadata_version, part->storage.getStorageID().getNameForLogs(), table_metadata_version);
continue;
}
/// StorageMergeTree does not have metadata version
if (part->storage.supportsReplication())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} with metadata version {} contains column {} that is absent "
"in table {} with metadata version {}",
part->name, part_metadata_version, column.name,
part->storage.getStorageID().getNameForLogs(), table_metadata_version);
}
for_interpreter.emplace_back(
@ -850,7 +852,7 @@ struct MutationContext
MergeTreeTransactionPtr txn;
MergeTreeData::HardlinkedFiles hardlinked_files;
HardlinkedFiles hardlinked_files;
bool need_prefix = true;
@ -1801,7 +1803,12 @@ bool MutateTask::prepare()
if (ctx->need_prefix)
prefix = "tmp_clone_";
auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks);
IDataPartStorage::ClonePartParams clone_params
{
.txn = ctx->txn, .hardlinked_files = &ctx->hardlinked_files,
.files_to_copy_instead_of_hardlinks = std::move(files_to_copy_instead_of_hardlinks), .keep_metadata_version = true
};
auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, clone_params);
part->getDataPartStorage().beginTransaction();
ctx->temporary_directory_lock = std::move(lock);
@ -1930,7 +1937,7 @@ bool MutateTask::prepare()
return true;
}
const MergeTreeData::HardlinkedFiles & MutateTask::getHardlinkedFiles() const
const HardlinkedFiles & MutateTask::getHardlinkedFiles() const
{
return ctx->hardlinked_files;
}

View File

@ -45,7 +45,7 @@ public:
return promise.get_future();
}
const MergeTreeData::HardlinkedFiles & getHardlinkedFiles() const;
const HardlinkedFiles & getHardlinkedFiles() const;
private:

View File

@ -181,6 +181,21 @@ bool RPNBuilderTreeNode::isConstant() const
}
}
bool RPNBuilderTreeNode::isSubqueryOrSet() const
{
if (ast_node)
{
return
typeid_cast<const ASTSubquery *>(ast_node) ||
typeid_cast<const ASTTableIdentifier *>(ast_node);
}
else
{
const auto * node_without_alias = getNodeWithoutAlias(dag_node);
return node_without_alias->result_type->getTypeId() == TypeIndex::Set;
}
}
ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const
{
if (!isConstant())

View File

@ -98,6 +98,8 @@ public:
/// Is node constant
bool isConstant() const;
bool isSubqueryOrSet() const;
/** Get constant as constant column.
* Node must be constant before calling these method, otherwise logical exception is thrown.
*/

View File

@ -631,7 +631,6 @@ template<bool async_insert>
void ReplicatedMergeTreeSinkImpl<async_insert>::writeExistingPart(MergeTreeData::MutableDataPartPtr & part)
{
/// NOTE: No delay in this case. That's Ok.
auto origin_zookeeper = storage.getZooKeeper();
assertSessionIsNotExpired(origin_zookeeper);
auto zookeeper = std::make_shared<ZooKeeperWithFaultInjection>(origin_zookeeper);

View File

@ -1932,7 +1932,8 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con
Int64 temp_index = insert_increment.get();
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, local_context->getCurrentTransaction(), {}, false, {});
IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()};
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot, clone_params);
dst_parts.emplace_back(std::move(dst_part));
dst_parts_locks.emplace_back(std::move(part_lock));
}
@ -2030,7 +2031,8 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const
Int64 temp_index = insert_increment.get();
MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level);
auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, local_context->getCurrentTransaction(), {}, false, {});
IDataPartStorage::ClonePartParams clone_params{.txn = local_context->getCurrentTransaction()};
auto [dst_part, part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params);
dst_parts.emplace_back(std::move(dst_part));
dst_parts_locks.emplace_back(std::move(part_lock));
}

View File

@ -1578,7 +1578,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
}
MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAndCommit(Transaction & transaction,
const MutableDataPartPtr & part, std::optional<MergeTreeData::HardlinkedFiles> hardlinked_files, bool replace_zero_copy_lock)
const MutableDataPartPtr & part, std::optional<HardlinkedFiles> hardlinked_files, bool replace_zero_copy_lock)
{
auto zookeeper = getZooKeeper();
@ -2183,7 +2183,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
/// A replica that will be used to fetch part
String replica;
MergeTreeData::HardlinkedFiles hardlinked_files;
HardlinkedFiles hardlinked_files;
scope_guard temporary_part_lock;
};
@ -2433,17 +2433,26 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
static const String TMP_PREFIX = "tmp_replace_from_";
std::vector<MergeTreeData::HardlinkedFiles> hardlinked_files_for_parts;
auto obtain_part = [&] (PartDescriptionPtr & part_desc)
{
if (part_desc->src_table_part)
/// Fetches with zero-copy-replication are cheap, but cloneAndLoadDataPartOnSameDisk will do full copy.
/// It's okay to check the setting for current table and disk for the source table, because src and dst part are on the same disk.
bool prefer_fetch_from_other_replica = !part_desc->replica.empty() && storage_settings_ptr->allow_remote_fs_zero_copy_replication
&& part_desc->src_table_part && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport();
if (part_desc->src_table_part && !prefer_fetch_from_other_replica)
{
if (part_desc->checksum_hex != part_desc->src_table_part->checksums.getTotalChecksumHex())
throw Exception(ErrorCodes::UNFINISHED, "Checksums of {} is suddenly changed", part_desc->src_table_part->name);
bool zero_copy_enabled = dynamic_cast<const MergeTreeData *>(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication;
IDataPartStorage::ClonePartParams clone_params
{
.copy_instead_of_hardlink = zero_copy_enabled && part_desc->src_table_part->isStoredOnRemoteDiskWithZeroCopySupport(),
.metadata_version_to_write = metadata_snapshot->getMetadataVersion()
};
auto [res_part, temporary_part_lock] = cloneAndLoadDataPartOnSameDisk(
part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, NO_TRANSACTION_PTR, &part_desc->hardlinked_files, false, {});
part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot, clone_params);
part_desc->res_part = std::move(res_part);
part_desc->temporary_part_lock = std::move(temporary_part_lock);
}
@ -4270,6 +4279,11 @@ bool StorageReplicatedMergeTree::fetchPart(
profile_events_scope.getSnapshot());
};
auto is_zero_copy_part = [&settings_ptr](const auto & data_part)
{
return settings_ptr->allow_remote_fs_zero_copy_replication && data_part->isStoredOnRemoteDiskWithZeroCopySupport();
};
DataPartPtr part_to_clone;
{
/// If the desired part is a result of a part mutation, try to find the source part and compare
@ -4281,7 +4295,7 @@ bool StorageReplicatedMergeTree::fetchPart(
auto source_part = getActiveContainingPart(covered_part_info);
/// Fetch for zero-copy replication is cheap and straightforward, so we don't use local clone here
if (source_part && (!settings_ptr->allow_remote_fs_zero_copy_replication || !source_part->getDataPartStorage().supportZeroCopyReplication()))
if (source_part && !is_zero_copy_part(source_part))
{
auto source_part_header = ReplicatedMergeTreePartHeader::fromColumnsAndChecksums(
source_part->getColumns(), source_part->checksums);
@ -4330,14 +4344,15 @@ bool StorageReplicatedMergeTree::fetchPart(
InterserverCredentialsPtr credentials;
std::optional<CurrentlySubmergingEmergingTagger> tagger_ptr;
std::function<MutableDataPartPtr()> get_part;
MergeTreeData::HardlinkedFiles hardlinked_files;
scope_guard part_directory_lock;
if (part_to_clone)
{
get_part = [&, part_to_clone]()
{
auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, false, {});
chassert(!is_zero_copy_part(part_to_clone));
IDataPartStorage::ClonePartParams clone_params{ .keep_metadata_version = true };
auto [cloned_part, lock] = cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot, clone_params);
part_directory_lock = std::move(lock);
return cloned_part;
};
@ -4387,7 +4402,8 @@ bool StorageReplicatedMergeTree::fetchPart(
Transaction transaction(*this, NO_TRANSACTION_RAW);
renameTempPartAndReplace(part, transaction);
replaced_parts = checkPartChecksumsAndCommit(transaction, part, hardlinked_files, !part_to_clone);
chassert(!part_to_clone || !is_zero_copy_part(part));
replaced_parts = checkPartChecksumsAndCommit(transaction, part, /*hardlinked_files*/ {}, /*replace_zero_copy_lock*/ true);
/** If a quorum is tracked for this part, you must update it.
* If you do not have time, in case of losing the session, when you restart the server - see the `ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart` method.
@ -7255,7 +7271,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
assert(replace == !LogEntry::ReplaceRangeEntry::isMovePartitionOrAttachFrom(drop_range));
String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range);
std::vector<MergeTreeData::HardlinkedFiles> hardlinked_files_for_parts;
for (const auto & src_part : src_all_parts)
{
@ -7286,19 +7301,21 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
UInt64 index = lock->getNumber();
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
MergeTreeData::HardlinkedFiles hardlinked_files;
bool copy_instead_of_hardlink = storage_settings_ptr->allow_remote_fs_zero_copy_replication
&& src_part->isStoredOnRemoteDiskWithZeroCopySupport();
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, copy_instead_of_hardlink, {});
bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication
|| dynamic_cast<const MergeTreeData *>(source_table.get())->getSettings()->allow_remote_fs_zero_copy_replication;
IDataPartStorage::ClonePartParams clone_params
{
.copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(),
.metadata_version_to_write = metadata_snapshot->getMetadataVersion()
};
auto [dst_part, part_lock] = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot, clone_params);
src_parts.emplace_back(src_part);
dst_parts.emplace_back(dst_part);
dst_parts_locks.emplace_back(std::move(part_lock));
ephemeral_locks.emplace_back(std::move(*lock));
block_id_paths.emplace_back(block_id_path);
part_checksums.emplace_back(hash_hex);
hardlinked_files_for_parts.emplace_back(hardlinked_files);
}
ReplicatedMergeTreeLogEntryData entry;
@ -7359,8 +7376,8 @@ void StorageReplicatedMergeTree::replacePartitionFrom(
renameTempPartAndReplaceUnlocked(part, transaction, data_parts_lock);
}
for (size_t i = 0; i < dst_parts.size(); ++i)
lockSharedData(*dst_parts[i], false, hardlinked_files_for_parts[i]);
for (const auto & dst_part : dst_parts)
lockSharedData(*dst_part, false, /*hardlinked_files*/ {});
Coordination::Error code = zookeeper->tryMulti(ops, op_results);
if (code == Coordination::Error::ZOK)
@ -7501,7 +7518,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
String dest_alter_partition_version_path = dest_table_storage->zookeeper_path + "/alter_partition_version";
Coordination::Stat dest_alter_partition_version_stat;
zookeeper->get(dest_alter_partition_version_path, &dest_alter_partition_version_stat);
std::vector<MergeTreeData::HardlinkedFiles> hardlinked_files_for_parts;
std::vector<scope_guard> temporary_parts_locks;
for (const auto & src_part : src_all_parts)
@ -7524,12 +7540,14 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
UInt64 index = lock->getNumber();
MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
MergeTreeData::HardlinkedFiles hardlinked_files;
bool copy_instead_of_hardlink = storage_settings_ptr->allow_remote_fs_zero_copy_replication
&& src_part->isStoredOnRemoteDiskWithZeroCopySupport();
auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, NO_TRANSACTION_PTR, &hardlinked_files, copy_instead_of_hardlink, {});
bool zero_copy_enabled = storage_settings_ptr->allow_remote_fs_zero_copy_replication
|| dynamic_cast<const MergeTreeData *>(dest_table.get())->getSettings()->allow_remote_fs_zero_copy_replication;
IDataPartStorage::ClonePartParams clone_params
{
.copy_instead_of_hardlink = zero_copy_enabled && src_part->isStoredOnRemoteDiskWithZeroCopySupport(),
.metadata_version_to_write = dest_metadata_snapshot->getMetadataVersion()
};
auto [dst_part, dst_part_lock] = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot, clone_params);
src_parts.emplace_back(src_part);
dst_parts.emplace_back(dst_part);
@ -7537,7 +7555,6 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
ephemeral_locks.emplace_back(std::move(*lock));
block_id_paths.emplace_back(block_id_path);
part_checksums.emplace_back(hash_hex);
hardlinked_files_for_parts.emplace_back(hardlinked_files);
}
ReplicatedMergeTreeLogEntryData entry_delete;
@ -7605,8 +7622,8 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta
for (auto & part : dst_parts)
dest_table_storage->renameTempPartAndReplaceUnlocked(part, transaction, dest_data_parts_lock);
for (size_t i = 0; i < dst_parts.size(); ++i)
dest_table_storage->lockSharedData(*dst_parts[i], false, hardlinked_files_for_parts[i]);
for (const auto & dst_part : dst_parts)
dest_table_storage->lockSharedData(*dst_part, false, /*hardlinked_files*/ {});
Coordination::Error code = zookeeper->tryMulti(ops, op_results);
if (code == Coordination::Error::ZBADVERSION)

View File

@ -184,7 +184,7 @@ public:
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override;
void onAddFilterFinish() override;
void applyFilters() override;
private:
void fillData(MutableColumns & res_columns);
@ -421,7 +421,7 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont
}
void ReadFromSystemZooKeeper::onAddFilterFinish()
void ReadFromSystemZooKeeper::applyFilters()
{
paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper);
}

View File

@ -0,0 +1,53 @@
<clickhouse>
<storage_configuration>
<disks>
<log_local>
<type>local</type>
<path>/var/lib/clickhouse/coordination/logs/</path>
</log_local>
<log_s3_plain>
<type>s3_plain</type>
<endpoint>http://minio1:9001/root/logs/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</log_s3_plain>
<snapshot_local>
<type>local</type>
<path>/var/lib/clickhouse/coordination/snapshots/</path>
</snapshot_local>
<snapshot_s3_plain>
<type>s3_plain</type>
<endpoint>http://minio1:9001/root/snapshots/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</snapshot_s3_plain>
</disks>
</storage_configuration>
<keeper_server>
<use_cluster>false</use_cluster>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<create_snapshot_on_exit>false</create_snapshot_on_exit>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>
<session_timeout_ms>10000</session_timeout_ms>
<raft_logs_level>trace</raft_logs_level>
<snapshot_distance>10</snapshot_distance>
<stale_log_gap>10</stale_log_gap>
<reserved_log_items>1</reserved_log_items>
<rotate_log_storage_interval>3</rotate_log_storage_interval>
</coordination_settings>
<!-- DISK DEFINITION PLACEHOLDER -->
<raft_configuration>
<server>
<id>1</id>
<hostname>node</hostname>
<port>9234</port>
</server>
</raft_configuration>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,258 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
from minio.deleteobjects import DeleteObject
import os
CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__))
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance(
"node", main_configs=["configs/enable_keeper.xml"], stay_alive=True, with_minio=True
)
from kazoo.client import KazooClient, KazooState
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
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 stop_zk(zk):
try:
if zk:
zk.stop()
zk.close()
except:
pass
def stop_clickhouse(cluster, cleanup_disks):
node.stop_clickhouse()
if not cleanup_disks:
return
node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/logs"])
node.exec_in_container(["rm", "-rf", "/var/lib/clickhouse/coordination/snapshots"])
s3_objects = list_s3_objects(cluster, prefix="")
if len(s3_objects) == 0:
return
assert (
len(
list(
cluster.minio_client.remove_objects(
cluster.minio_bucket,
[DeleteObject(obj) for obj in s3_objects],
)
)
)
== 0
)
def setup_storage(cluster, storage_config, cleanup_disks):
stop_clickhouse(cluster, cleanup_disks)
node.copy_file_to_container(
os.path.join(CURRENT_TEST_DIR, "configs/enable_keeper.xml"),
"/etc/clickhouse-server/config.d/enable_keeper.xml",
)
node.replace_in_config(
"/etc/clickhouse-server/config.d/enable_keeper.xml",
"<!-- DISK DEFINITION PLACEHOLDER -->",
storage_config,
)
node.start_clickhouse()
keeper_utils.wait_until_connected(cluster, node)
def setup_local_storage(cluster):
setup_storage(
cluster,
"<log_storage_disk>log_local<\\/log_storage_disk>"
"<snapshot_storage_disk>snapshot_local<\\/snapshot_storage_disk>",
cleanup_disks=True,
)
def list_s3_objects(cluster, prefix=""):
minio = cluster.minio_client
prefix_len = len(prefix)
return [
obj.object_name[prefix_len:]
for obj in minio.list_objects(
cluster.minio_bucket, prefix=prefix, recursive=True
)
]
def get_local_files(path):
files = node.exec_in_container(["ls", path]).strip().split("\n")
files.sort()
return files
def get_local_logs():
return get_local_files("/var/lib/clickhouse/coordination/logs")
def get_local_snapshots():
return get_local_files("/var/lib/clickhouse/coordination/snapshots")
def test_logs_with_disks(started_cluster):
setup_local_storage(started_cluster)
node_zk = get_fake_zk("node")
try:
node_zk.create("/test")
for _ in range(30):
node_zk.create("/test/somenode", b"somedata", sequence=True)
stop_zk(node_zk)
previous_log_files = get_local_logs()
setup_storage(
started_cluster,
"<log_storage_disk>log_s3_plain<\\/log_storage_disk>"
"<latest_log_storage_disk>log_local<\\/latest_log_storage_disk>"
"<snapshot_storage_disk>snapshot_local<\\/snapshot_storage_disk>",
cleanup_disks=False,
)
# all but the latest log should be on S3
s3_log_files = list_s3_objects(started_cluster, "logs/")
assert set(s3_log_files) == set(previous_log_files[:-1])
local_log_files = get_local_logs()
assert len(local_log_files) == 1
assert local_log_files[0] == previous_log_files[-1]
previous_log_files = s3_log_files + local_log_files
node_zk = get_fake_zk("node")
for _ in range(30):
node_zk.create("/test/somenode", b"somedata", sequence=True)
stop_zk(node_zk)
log_files = list_s3_objects(started_cluster, "logs/")
local_log_files = get_local_logs()
assert len(local_log_files) == 1
log_files.extend(local_log_files)
assert set(log_files) != previous_log_files
previous_log_files = log_files
setup_storage(
started_cluster,
"<old_log_storage_disk>log_s3_plain<\\/old_log_storage_disk>"
"<log_storage_disk>log_local<\\/log_storage_disk>"
"<snapshot_storage_disk>snapshot_local<\\/snapshot_storage_disk>",
cleanup_disks=False,
)
local_log_files = get_local_logs()
assert set(local_log_files) == set(previous_log_files)
node_zk = get_fake_zk("node")
for child in node_zk.get_children("/test"):
assert node_zk.get(f"/test/{child}")[0] == b"somedata"
finally:
stop_zk(node_zk)
def test_snapshots_with_disks(started_cluster):
setup_local_storage(started_cluster)
node_zk = get_fake_zk("node")
try:
node_zk.create("/test2")
for _ in range(30):
node_zk.create("/test2/somenode", b"somedata", sequence=True)
stop_zk(node_zk)
snapshot_idx = keeper_utils.send_4lw_cmd(cluster, node, "csnp")
node.wait_for_log_line(
f"Created persistent snapshot {snapshot_idx}", look_behind_lines=1000
)
previous_snapshot_files = get_local_snapshots()
setup_storage(
started_cluster,
"<snapshot_storage_disk>snapshot_s3_plain<\\/snapshot_storage_disk>"
"<latest_snapshot_storage_disk>snapshot_local<\\/latest_snapshot_storage_disk>"
"<log_storage_disk>log_local<\\/log_storage_disk>",
cleanup_disks=False,
)
## all but the latest log should be on S3
s3_snapshot_files = list_s3_objects(started_cluster, "snapshots/")
assert set(s3_snapshot_files) == set(previous_snapshot_files[:-1])
local_snapshot_files = get_local_snapshots()
assert len(local_snapshot_files) == 1
assert local_snapshot_files[0] == previous_snapshot_files[-1]
previous_snapshot_files = s3_snapshot_files + local_snapshot_files
node_zk = get_fake_zk("node")
for _ in range(30):
node_zk.create("/test2/somenode", b"somedata", sequence=True)
stop_zk(node_zk)
snapshot_idx = keeper_utils.send_4lw_cmd(cluster, node, "csnp")
node.wait_for_log_line(
f"Created persistent snapshot {snapshot_idx}", look_behind_lines=1000
)
snapshot_files = list_s3_objects(started_cluster, "snapshots/")
local_snapshot_files = get_local_snapshots()
assert len(local_snapshot_files) == 1
snapshot_files.extend(local_snapshot_files)
previous_snapshot_files = snapshot_files
setup_storage(
started_cluster,
"<old_snapshot_storage_disk>snapshot_s3_plain<\\/old_snapshot_storage_disk>"
"<snapshot_storage_disk>snapshot_local<\\/snapshot_storage_disk>"
"<log_storage_disk>log_local<\\/log_storage_disk>",
cleanup_disks=False,
)
local_snapshot_files = get_local_snapshots()
assert set(local_snapshot_files) == set(previous_snapshot_files)
node_zk = get_fake_zk("node")
for child in node_zk.get_children("/test2"):
assert node_zk.get(f"/test2/{child}")[0] == b"somedata"
finally:
stop_zk(node_zk)

View File

@ -252,10 +252,12 @@ def test_cmd_conf(started_cluster):
assert result["four_letter_word_allow_list"] == "*"
assert result["log_storage_path"] == "/var/lib/clickhouse/coordination/log"
assert result["log_storage_disk"] == "LocalLogDisk"
assert (
result["snapshot_storage_path"]
== "/var/lib/clickhouse/coordination/snapshots"
)
assert result["snapshot_storage_disk"] == "LocalSnapshotDisk"
assert result["session_timeout_ms"] == "30000"
assert result["min_session_timeout_ms"] == "10000"

View File

@ -0,0 +1,26 @@
<clickhouse>
<storage_configuration>
<disks>
<snapshot_s3_plain1>
<type>s3_plain</type>
<endpoint>http://minio1:9001/root/data/snapshots1/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</snapshot_s3_plain1>
<snapshot_s3_plain2>
<type>s3_plain</type>
<endpoint>http://minio1:9001/root/data/snapshots2/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</snapshot_s3_plain2>
<snapshot_s3_plain3>
<type>s3_plain</type>
<endpoint>http://minio1:9001/root/data/snapshots3/</endpoint>
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</snapshot_s3_plain3>
</disks>
</storage_configuration>
</clickhouse>

View File

@ -2,8 +2,6 @@
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>

View File

@ -2,8 +2,6 @@
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>

View File

@ -2,8 +2,6 @@
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
<coordination_settings>
<operation_timeout_ms>5000</operation_timeout_ms>

View File

@ -0,0 +1,5 @@
<clickhouse>
<keeper_server>
<snapshot_storage_disk>snapshot_s3_plain1</snapshot_storage_disk>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,5 @@
<clickhouse>
<keeper_server>
<snapshot_storage_disk>snapshot_s3_plain2</snapshot_storage_disk>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,5 @@
<clickhouse>
<keeper_server>
<snapshot_storage_disk>snapshot_s3_plain3</snapshot_storage_disk>
</keeper_server>
</clickhouse>

View File

@ -0,0 +1,6 @@
<clickhouse>
<keeper_server>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
<snapshot_storage_path>/var/lib/clickhouse/coordination/snapshots</snapshot_storage_path>
</keeper_server>
</clickhouse>

View File

@ -9,13 +9,19 @@ import time
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True
"node1",
main_configs=["configs/enable_keeper1.xml", "configs/local_storage_path.xml"],
stay_alive=True,
)
node2 = cluster.add_instance(
"node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True
"node2",
main_configs=["configs/enable_keeper2.xml", "configs/local_storage_path.xml"],
stay_alive=True,
)
node3 = cluster.add_instance(
"node3", main_configs=["configs/enable_keeper3.xml"], stay_alive=True
"node3",
main_configs=["configs/enable_keeper3.xml", "configs/local_storage_path.xml"],
stay_alive=True,
)
from kazoo.client import KazooClient, KazooState

View File

@ -0,0 +1,152 @@
#!/usr/bin/env python3
import pytest
from helpers.cluster import ClickHouseCluster
import helpers.keeper_utils as keeper_utils
import random
import string
import os
import time
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1",
main_configs=[
"configs/enable_keeper1.xml",
"configs/disk_s3_storage.xml",
"configs/keeper1_snapshot_disk.xml",
],
stay_alive=True,
with_minio=True,
)
node2 = cluster.add_instance(
"node2",
main_configs=[
"configs/enable_keeper2.xml",
"configs/disk_s3_storage.xml",
"configs/keeper2_snapshot_disk.xml",
],
stay_alive=True,
with_minio=True,
)
node3 = cluster.add_instance(
"node3",
main_configs=[
"configs/enable_keeper3.xml",
"configs/disk_s3_storage.xml",
"configs/keeper3_snapshot_disk.xml",
],
stay_alive=True,
with_minio=True,
)
from kazoo.client import KazooClient, KazooState
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
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 stop_zk(zk):
try:
if zk:
zk.stop()
zk.close()
except:
pass
def test_recover_from_snapshot_with_disk_s3(started_cluster):
try:
node1_zk = node2_zk = node3_zk = None
node1_zk = get_fake_zk("node1")
node2_zk = get_fake_zk("node2")
node3_zk = get_fake_zk("node3")
node1_zk.create("/test_snapshot_multinode_recover", "somedata".encode())
node2_zk.sync("/test_snapshot_multinode_recover")
node3_zk.sync("/test_snapshot_multinode_recover")
assert node1_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata"
assert node2_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata"
assert node3_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata"
node3.stop_clickhouse(kill=True)
# at least we will have 2 snapshots
for i in range(435):
node1_zk.create(
"/test_snapshot_multinode_recover" + str(i),
("somedata" + str(i)).encode(),
)
for i in range(435):
if i % 10 == 0:
node1_zk.delete("/test_snapshot_multinode_recover" + str(i))
finally:
for zk in [node1_zk, node2_zk, node3_zk]:
stop_zk(zk)
# stale node should recover from leader's snapshot
# with some sanitizers can start longer than 5 seconds
node3.start_clickhouse(20)
keeper_utils.wait_until_connected(cluster, node3)
print("Restarted")
try:
node1_zk = node2_zk = node3_zk = None
node1_zk = get_fake_zk("node1")
node2_zk = get_fake_zk("node2")
node3_zk = get_fake_zk("node3")
node1_zk.sync("/test_snapshot_multinode_recover")
node2_zk.sync("/test_snapshot_multinode_recover")
node3_zk.sync("/test_snapshot_multinode_recover")
assert node1_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata"
assert node2_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata"
assert node3_zk.get("/test_snapshot_multinode_recover")[0] == b"somedata"
for i in range(435):
if i % 10 != 0:
assert (
node1_zk.get("/test_snapshot_multinode_recover" + str(i))[0]
== ("somedata" + str(i)).encode()
)
assert (
node2_zk.get("/test_snapshot_multinode_recover" + str(i))[0]
== ("somedata" + str(i)).encode()
)
assert (
node3_zk.get("/test_snapshot_multinode_recover" + str(i))[0]
== ("somedata" + str(i)).encode()
)
else:
assert (
node1_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None
)
assert (
node2_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None
)
assert (
node3_zk.exists("/test_snapshot_multinode_recover" + str(i)) is None
)
finally:
for zk in [node1_zk, node2_zk, node3_zk]:
stop_zk(zk)

View File

@ -235,12 +235,7 @@ def test_attach_detach_partition(cluster):
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
hdfs_objects = fs.listdir("/clickhouse")
assert (
len(hdfs_objects)
== FILES_OVERHEAD
+ FILES_OVERHEAD_PER_PART_WIDE * 2
- FILES_OVERHEAD_METADATA_VERSION
)
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
node.query("ALTER TABLE hdfs_test DROP PARTITION '2020-01-03'")
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)"

View File

@ -336,9 +336,7 @@ def test_attach_detach_partition(cluster, node_name):
assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(8192)"
assert (
len(list_objects(cluster, "data/"))
== FILES_OVERHEAD
+ FILES_OVERHEAD_PER_PART_WIDE * 2
- FILES_OVERHEAD_METADATA_VERSION
== FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
)
node.query("ALTER TABLE s3_test DROP PARTITION '2020-01-03'")

View File

@ -8,6 +8,28 @@
<size>never</size>
</logger>
<storage_configuration>
<disks>
<log_local>
<type>local</type>
<path>/home/robot-clickhouse/db/coordination/logs/</path>
</log_local>
<latest_log_local>
<type>local</type>
<path>/home/robot-clickhouse/db/coordination/latest_log/</path>
</latest_log_local>
<snapshot_local>
<type>local</type>
<path>/home/robot-clickhouse/db/coordination/snapshots/</path>
</snapshot_local>
<latest_snapshot_local>
<type>local</type>
<path>/home/robot-clickhouse/db/coordination/latest_snapshot/</path>
</latest_snapshot_local>
</disks>
</storage_configuration>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>{id}</server_id>

View File

@ -14,7 +14,9 @@
(def coordination-data-dir (str data-dir "/coordination"))
(def coordination-snapshots-dir (str coordination-data-dir "/snapshots"))
(def coordination-latest-snapshot-dir (str coordination-data-dir "/latest_snapshot"))
(def coordination-logs-dir (str coordination-data-dir "/logs"))
(def coordination-latest_log-dir (str coordination-data-dir "/latest_log"))
(def stderr-file (str logs-dir "/stderr.log"))

View File

@ -146,6 +146,8 @@
:--
:--logger.log (str logs-dir "/clickhouse-keeper.log")
:--logger.errorlog (str logs-dir "/clickhouse-keeper.err.log")
:--keeper_server.snapshot_storage_path coordination-snapshots-dir
:--keeper_server.log_storage_path coordination-logs-dir
:--keeper_server.snapshot_storage_disk "snapshot_local"
:--keeper_server.latest_snapshot_storage_disk "latest_snapshot_local"
:--keeper_server.log_storage_disk "log_local"
:--keeper_server.latest_log_storage_disk "latest_log_local"
:--path coordination-data-dir))

View File

@ -12,7 +12,7 @@ db="test_01193_$RANDOM"
tables=1000
threads=10
count_multiplier=1
max_time_ms=5000
max_time_ms=1500
debug_or_sanitizer_build=$($CLICKHOUSE_CLIENT -q "WITH ((SELECT value FROM system.build_options WHERE name='BUILD_TYPE') AS build, (SELECT value FROM system.build_options WHERE name='CXX_FLAGS') as flags) SELECT build='Debug' OR flags LIKE '%fsanitize%' OR hasThreadFuzzer()")
@ -42,13 +42,15 @@ wait
$CLICKHOUSE_CLIENT -q "CREATE TABLE $db.table_merge (i UInt64, d Date, s String, n Nested(i UInt8, f Float32)) ENGINE=Merge('$db', '^table_')"
$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM merge('$db', '^table_9') GROUP BY i, d, s, n.i, n.f ORDER BY i"
for i in {1..10}; do
for i in {1..50}; do
$CLICKHOUSE_CLIENT -q "DETACH DATABASE $db"
$CLICKHOUSE_CLIENT -q "ATTACH DATABASE $db" --query_id="$db-$i";
$CLICKHOUSE_CLIENT --query_profiler_real_time_period_ns=100000000 --query_profiler_cpu_time_period_ns=100000000 -q "ATTACH DATABASE $db" --query_id="$db-$i";
done
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT -q "SELECT if(min(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2"
durations=$($CLICKHOUSE_CLIENT -q "SELECT groupArray(query_duration_ms) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2")
$CLICKHOUSE_CLIENT -q "SELECT 'durations', '$db', $durations FORMAT Null"
$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(arrayJoin($durations)) < $max_time_ms, 'ok', toString($durations))"
$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i"

View File

@ -0,0 +1,8 @@
PREWHERE a IN
PREWHERE a IN
PREWHERE a IN
PREWHERE a IN
PREWHERE b NOT IN
PREWHERE b NOT IN
PREWHERE b NOT IN
PREWHERE b NOT IN

View File

@ -0,0 +1,56 @@
DROP TABLE IF EXISTS t_02809;
CREATE TABLE t_02809(a Int64, b Int64, s String)
ENGINE=MergeTree order by tuple()
AS SELECT number, number%10, toString(arrayMap(i-> cityHash64(i*number), range(50))) FROM numbers(10000);
CREATE TABLE t_02809_set(c Int64)
ENGINE=Set()
AS SELECT * FROM numbers(10);
CREATE TABLE t_02809_aux(c Int64)
ENGINE=Memory()
AS SELECT * FROM numbers(10);
SET optimize_move_to_prewhere=1;
-- Queries with 'IN'
SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE a IN (SELECT * FROM system.one)
) WHERE explain LIKE '%WHERE%';
SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE a IN (1,2,3)
) WHERE explain LIKE '%WHERE%';
SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE a IN t_02809_set
) WHERE explain LIKE '%WHERE%';
SELECT substring(explain, 1, 13) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE a IN t_02809_aux
) WHERE explain LIKE '%WHERE%';
-- Queries with 'NOT IN'
SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE b NOT IN (SELECT * FROM system.one)
) WHERE explain LIKE '%WHERE%';
SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE b NOT IN (1,2,3)
) WHERE explain LIKE '%WHERE%';
SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE b NOT IN t_02809_set
) WHERE explain LIKE '%WHERE%';
SELECT substring(explain, 1, 17) FROM (EXPLAIN SYNTAX
SELECT * FROM t_02809 WHERE b NOT IN t_02809_aux
) WHERE explain LIKE '%WHERE%';
DROP TABLE t_02809;
DROP TABLE t_02809_set;
DROP TABLE t_02809_aux;

View File

@ -8,6 +8,7 @@
#include <Coordination/KeeperLogStore.h>
#include <Coordination/Changelog.h>
#include <Common/logger_useful.h>
#include <Disks/DiskLocal.h>
using namespace Coordination;
using namespace DB;
@ -62,15 +63,18 @@ int main(int argc, char *argv[])
ResponsesQueue queue(std::numeric_limits<size_t>::max());
SnapshotsQueue snapshots_queue{1};
CoordinationSettingsPtr settings = std::make_shared<CoordinationSettings>();
KeeperContextPtr keeper_context = std::make_shared<DB::KeeperContext>();
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, argv[1], settings, keeper_context, nullptr);
KeeperContextPtr keeper_context = std::make_shared<DB::KeeperContext>(true);
keeper_context->setLogDisk(std::make_shared<DB::DiskLocal>("LogDisk", argv[2], 0));
keeper_context->setSnapshotDisk(std::make_shared<DB::DiskLocal>("LogDisk", argv[1], 0));
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, settings, keeper_context, nullptr);
state_machine->init();
size_t last_commited_index = state_machine->last_commit_index();
LOG_INFO(logger, "Last committed index: {}", last_commited_index);
DB::KeeperLogStore changelog(
argv[2], LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000});
LogFileSettings{.force_sync = true, .compress_logs = settings->compress_logs, .rotate_interval = 10000000}, keeper_context);
changelog.init(last_commited_index, 10000000000UL); /// collect all logs
if (changelog.size() == 0)
LOG_INFO(logger, "Changelog empty");