mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge remote-tracking branch 'origin' into integration-2
This commit is contained in:
commit
28af06f3d0
@ -13,3 +13,6 @@ ClickHouse® is an open-source column-oriented database management system that a
|
||||
* [Code Browser](https://clickhouse.tech/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation.
|
||||
* [Contacts](https://clickhouse.tech/#contacts) can help to get your questions answered if there are any.
|
||||
* You can also [fill this form](https://clickhouse.tech/#meet) to meet Yandex ClickHouse team in person.
|
||||
|
||||
## Upcoming Events
|
||||
* [SF Bay Area ClickHouse Community Meetup (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/278144089/) on 16 June 2021.
|
||||
|
@ -79,7 +79,8 @@ RUN python3 -m pip install \
|
||||
redis \
|
||||
tzlocal \
|
||||
urllib3 \
|
||||
requests-kerberos
|
||||
requests-kerberos \
|
||||
pyhdfs
|
||||
|
||||
COPY modprobe.sh /usr/local/bin/modprobe
|
||||
COPY dockerd-entrypoint.sh /usr/local/bin/
|
||||
|
@ -199,7 +199,7 @@ Provides possibility to start background move data according to [TTL table expre
|
||||
Returns `Ok.` even if table doesn’t exist. Returns error when database doesn’t exist:
|
||||
|
||||
``` sql
|
||||
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
|
||||
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
|
||||
```
|
||||
|
||||
## Managing ReplicatedMergeTree Tables {#query-language-system-replicated}
|
||||
|
@ -101,6 +101,7 @@ endif()
|
||||
|
||||
if (USE_HDFS)
|
||||
add_headers_and_sources(dbms Storages/HDFS)
|
||||
add_headers_and_sources(dbms Disks/HDFS)
|
||||
endif()
|
||||
|
||||
list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD})
|
||||
|
@ -113,12 +113,22 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
|
||||
if (shutdown)
|
||||
return on_error();
|
||||
|
||||
jobs.emplace(std::move(job), priority);
|
||||
++scheduled_jobs;
|
||||
/// We must not to allocate any memory after we emplaced a job in a queue.
|
||||
/// Because if an exception would be thrown, we won't notify a thread about job occurrence.
|
||||
|
||||
if (threads.size() < std::min(max_threads, scheduled_jobs))
|
||||
/// Check if there are enough threads to process job.
|
||||
if (threads.size() < std::min(max_threads, scheduled_jobs + 1))
|
||||
{
|
||||
threads.emplace_front();
|
||||
try
|
||||
{
|
||||
threads.emplace_front();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Most likely this is a std::bad_alloc exception
|
||||
return on_error();
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
|
||||
@ -126,19 +136,15 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, int priority, std::opti
|
||||
catch (...)
|
||||
{
|
||||
threads.pop_front();
|
||||
|
||||
/// Remove the job and return error to caller.
|
||||
/// Note that if we have allocated at least one thread, we may continue
|
||||
/// (one thread is enough to process all jobs).
|
||||
/// But this condition indicate an error nevertheless and better to refuse.
|
||||
|
||||
jobs.pop();
|
||||
--scheduled_jobs;
|
||||
return on_error();
|
||||
}
|
||||
}
|
||||
|
||||
jobs.emplace(std::move(job), priority);
|
||||
++scheduled_jobs;
|
||||
new_job_or_shutdown.notify_one();
|
||||
}
|
||||
new_job_or_shutdown.notify_one();
|
||||
|
||||
return ReturnType(true);
|
||||
}
|
||||
|
||||
@ -165,6 +171,10 @@ void ThreadPoolImpl<Thread>::wait()
|
||||
{
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
/// Signal here just in case.
|
||||
/// If threads are waiting on condition variables, but there are some jobs in the queue
|
||||
/// then it will prevent us from deadlock.
|
||||
new_job_or_shutdown.notify_all();
|
||||
job_finished.wait(lock, [this] { return scheduled_jobs == 0; });
|
||||
|
||||
if (first_exception)
|
||||
|
@ -18,7 +18,6 @@ PEERDIR(
|
||||
contrib/libs/openssl
|
||||
contrib/libs/poco/NetSSL_OpenSSL
|
||||
contrib/libs/re2
|
||||
contrib/libs/cxxsupp/libcxxabi-parts
|
||||
contrib/restricted/dragonbox
|
||||
)
|
||||
|
||||
|
@ -17,7 +17,6 @@ PEERDIR(
|
||||
contrib/libs/openssl
|
||||
contrib/libs/poco/NetSSL_OpenSSL
|
||||
contrib/libs/re2
|
||||
contrib/libs/cxxsupp/libcxxabi-parts
|
||||
contrib/restricted/dragonbox
|
||||
)
|
||||
|
||||
|
@ -139,7 +139,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
writeBinary(snapshot.session_id, out);
|
||||
writeBinary(snapshot.snapshot_container_size, out);
|
||||
size_t counter = 0;
|
||||
for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++it, ++counter)
|
||||
for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter)
|
||||
{
|
||||
const auto & path = it->key;
|
||||
const auto & node = it->value;
|
||||
@ -148,6 +148,13 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
|
||||
|
||||
writeBinary(path, out);
|
||||
writeNode(node, out);
|
||||
|
||||
/// Last iteration: check and exit here without iterator increment. Otherwise
|
||||
/// false positive race condition on list end is possible.
|
||||
if (counter == snapshot.snapshot_container_size - 1)
|
||||
break;
|
||||
|
||||
++it;
|
||||
}
|
||||
|
||||
size_t size = snapshot.session_and_timeout.size();
|
||||
|
@ -116,6 +116,12 @@ template <typename A, typename B> struct ResultOfModulo
|
||||
using Type = std::conditional_t<std::is_floating_point_v<A> || std::is_floating_point_v<B>, Float64, Type0>;
|
||||
};
|
||||
|
||||
template <typename A, typename B> struct ResultOfModuloLegacy
|
||||
{
|
||||
using Type0 = typename Construct<is_signed_v<A> || is_signed_v<B>, false, sizeof(B)>::Type;
|
||||
using Type = std::conditional_t<std::is_floating_point_v<A> || std::is_floating_point_v<B>, Float64, Type0>;
|
||||
};
|
||||
|
||||
template <typename A> struct ResultOfNegate
|
||||
{
|
||||
using Type = typename Construct<
|
||||
|
@ -11,7 +11,8 @@ struct DiskType
|
||||
{
|
||||
Local,
|
||||
RAM,
|
||||
S3
|
||||
S3,
|
||||
HDFS
|
||||
};
|
||||
static String toString(Type disk_type)
|
||||
{
|
||||
@ -23,10 +24,11 @@ struct DiskType
|
||||
return "memory";
|
||||
case Type::S3:
|
||||
return "s3";
|
||||
case Type::HDFS:
|
||||
return "hdfs";
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
194
src/Disks/HDFS/DiskHDFS.cpp
Normal file
194
src/Disks/HDFS/DiskHDFS.cpp
Normal file
@ -0,0 +1,194 @@
|
||||
#include <Disks/HDFS/DiskHDFS.h>
|
||||
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Disks/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
class HDFSPathKeeper : public RemoteFSPathKeeper
|
||||
{
|
||||
public:
|
||||
using Chunk = std::vector<std::string>;
|
||||
using Chunks = std::list<Chunk>;
|
||||
|
||||
explicit HDFSPathKeeper(size_t chunk_limit_) : RemoteFSPathKeeper(chunk_limit_) {}
|
||||
|
||||
void addPath(const String & path) override
|
||||
{
|
||||
if (chunks.empty() || chunks.back().size() >= chunk_limit)
|
||||
{
|
||||
chunks.push_back(Chunks::value_type());
|
||||
chunks.back().reserve(chunk_limit);
|
||||
}
|
||||
chunks.back().push_back(path.data());
|
||||
}
|
||||
|
||||
void removePaths(const std::function<void(Chunk &&)> & remove_chunk_func)
|
||||
{
|
||||
for (auto & chunk : chunks)
|
||||
remove_chunk_func(std::move(chunk));
|
||||
}
|
||||
|
||||
private:
|
||||
Chunks chunks;
|
||||
};
|
||||
|
||||
|
||||
/// Reads data from HDFS using stored paths in metadata.
|
||||
class ReadIndirectBufferFromHDFS final : public ReadIndirectBufferFromRemoteFS<ReadBufferFromHDFS>
|
||||
{
|
||||
public:
|
||||
ReadIndirectBufferFromHDFS(
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
const String & hdfs_uri_,
|
||||
DiskHDFS::Metadata metadata_,
|
||||
size_t buf_size_)
|
||||
: ReadIndirectBufferFromRemoteFS<ReadBufferFromHDFS>(metadata_)
|
||||
, config(config_)
|
||||
, buf_size(buf_size_)
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2);
|
||||
hdfs_directory = hdfs_uri_.substr(begin_of_path);
|
||||
hdfs_uri = hdfs_uri_.substr(0, begin_of_path);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromHDFS> createReadBuffer(const String & path) override
|
||||
{
|
||||
return std::make_unique<ReadBufferFromHDFS>(hdfs_uri, hdfs_directory + path, config, buf_size);
|
||||
}
|
||||
|
||||
private:
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
String hdfs_uri;
|
||||
String hdfs_directory;
|
||||
size_t buf_size;
|
||||
};
|
||||
|
||||
|
||||
DiskHDFS::DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
const String & metadata_path_,
|
||||
const Poco::Util::AbstractConfiguration & config_)
|
||||
: IDiskRemote(disk_name_, hdfs_root_path_, metadata_path_, "DiskHDFS", settings_->thread_pool_size)
|
||||
, config(config_)
|
||||
, hdfs_builder(createHDFSBuilder(hdfs_root_path_, config))
|
||||
, hdfs_fs(createHDFSFS(hdfs_builder.get()))
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskHDFS::readFile(const String & path, size_t buf_size, size_t, size_t, size_t, MMappedFileCache *) const
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
|
||||
LOG_DEBUG(log,
|
||||
"Read from file by path: {}. Existing HDFS objects: {}",
|
||||
backQuote(metadata_path + path), metadata.remote_fs_objects.size());
|
||||
|
||||
auto reader = std::make_unique<ReadIndirectBufferFromHDFS>(config, remote_fs_root_path, metadata, buf_size);
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(reader), settings->min_bytes_for_seek);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode)
|
||||
{
|
||||
auto metadata = readOrCreateMetaForWriting(path, mode);
|
||||
|
||||
/// Path to store new HDFS object.
|
||||
auto file_name = getRandomName();
|
||||
auto hdfs_path = remote_fs_root_path + file_name;
|
||||
|
||||
LOG_DEBUG(log, "{} to file by path: {}. HDFS path: {}", mode == WriteMode::Rewrite ? "Write" : "Append",
|
||||
backQuote(metadata_path + path), remote_fs_root_path + hdfs_path);
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(hdfs_path,
|
||||
config, buf_size,
|
||||
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS<WriteBufferFromHDFS>>(std::move(hdfs_buffer),
|
||||
std::move(metadata),
|
||||
file_name);
|
||||
}
|
||||
|
||||
|
||||
RemoteFSPathKeeperPtr DiskHDFS::createFSPathKeeper() const
|
||||
{
|
||||
return std::make_shared<HDFSPathKeeper>(settings->objects_chunk_size_to_delete);
|
||||
}
|
||||
|
||||
|
||||
void DiskHDFS::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper)
|
||||
{
|
||||
auto * hdfs_paths_keeper = dynamic_cast<HDFSPathKeeper *>(fs_paths_keeper.get());
|
||||
if (hdfs_paths_keeper)
|
||||
hdfs_paths_keeper->removePaths([&](std::vector<std::string> && chunk)
|
||||
{
|
||||
for (const auto & hdfs_object_path : chunk)
|
||||
{
|
||||
const String & hdfs_path = hdfs_object_path;
|
||||
const size_t begin_of_path = hdfs_path.find('/', hdfs_path.find("//") + 2);
|
||||
|
||||
/// Add path from root to file name
|
||||
int res = hdfsDelete(hdfs_fs.get(), hdfs_path.substr(begin_of_path).c_str(), 0);
|
||||
if (res == -1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "HDFSDelete failed with path: " + hdfs_path);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
std::unique_ptr<DiskHDFSSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
return std::make_unique<DiskHDFSSettings>(
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000));
|
||||
}
|
||||
}
|
||||
|
||||
void registerDiskHDFS(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextConstPtr context_) -> DiskPtr
|
||||
{
|
||||
Poco::File disk{context_->getPath() + "disks/" + name};
|
||||
disk.createDirectories();
|
||||
|
||||
String uri{config.getString(config_prefix + ".endpoint")};
|
||||
|
||||
if (uri.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
|
||||
|
||||
String metadata_path = context_->getPath() + "disks/" + name + "/";
|
||||
|
||||
return std::make_shared<DiskHDFS>(
|
||||
name, uri,
|
||||
getSettings(config, config_prefix),
|
||||
metadata_path, config);
|
||||
};
|
||||
|
||||
factory.registerDiskType("hdfs", creator);
|
||||
}
|
||||
|
||||
}
|
72
src/Disks/HDFS/DiskHDFS.h
Normal file
72
src/Disks/HDFS/DiskHDFS.h
Normal file
@ -0,0 +1,72 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct DiskHDFSSettings
|
||||
{
|
||||
size_t min_bytes_for_seek;
|
||||
int thread_pool_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
|
||||
DiskHDFSSettings(
|
||||
int min_bytes_for_seek_,
|
||||
int thread_pool_size_,
|
||||
int objects_chunk_size_to_delete_)
|
||||
: min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, thread_pool_size(thread_pool_size_)
|
||||
, objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {}
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in HDFS and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains HDFS object key with actual data.
|
||||
*/
|
||||
class DiskHDFS final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using SettingsPtr = std::unique_ptr<DiskHDFSSettings>;
|
||||
|
||||
DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
const String & metadata_path_,
|
||||
const Poco::Util::AbstractConfiguration & config_);
|
||||
|
||||
DiskType::Type getType() const override { return DiskType::Type::HDFS; }
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
size_t estimated_size,
|
||||
size_t aio_threshold,
|
||||
size_t mmap_threshold,
|
||||
MMappedFileCache * mmap_cache) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String & path, size_t buf_size, WriteMode mode) override;
|
||||
|
||||
void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) override;
|
||||
|
||||
RemoteFSPathKeeperPtr createFSPathKeeper() const override;
|
||||
|
||||
private:
|
||||
String getRandomName() { return toString(UUIDHelpers::generateV4()); }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
|
||||
HDFSBuilderWrapper hdfs_builder;
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
};
|
||||
|
||||
}
|
@ -7,6 +7,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Disks/Executor.h>
|
||||
#include <Disks/DiskType.h>
|
||||
#include "Disks/Executor.h"
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
@ -178,17 +179,17 @@ public:
|
||||
virtual void removeRecursive(const String & path) = 0;
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or if directory is not empty.
|
||||
/// Differs from removeFile for S3 disks
|
||||
/// Differs from removeFile for S3/HDFS disks
|
||||
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
|
||||
virtual void removeSharedFile(const String & path, bool) { removeFile(path); }
|
||||
|
||||
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
|
||||
/// Differs from removeRecursive for S3 disks
|
||||
/// Differs from removeRecursive for S3/HDFS disks
|
||||
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
|
||||
virtual void removeSharedRecursive(const String & path, bool) { removeRecursive(path); }
|
||||
|
||||
/// Remove file or directory if it exists.
|
||||
/// Differs from removeFileIfExists for S3 disks
|
||||
/// Differs from removeFileIfExists for S3/HDFS disks
|
||||
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
|
||||
virtual void removeSharedFileIfExists(const String & path, bool) { removeFileIfExists(path); }
|
||||
|
||||
|
487
src/Disks/IDiskRemote.cpp
Normal file
487
src/Disks/IDiskRemote.cpp
Normal file
@ -0,0 +1,487 @@
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/createHardLink.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
extern const int PATH_ACCESS_DENIED;;
|
||||
extern const int CANNOT_DELETE_DIRECTORY;
|
||||
}
|
||||
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
IDiskRemote::Metadata::Metadata(
|
||||
const String & remote_fs_root_path_,
|
||||
const String & disk_path_,
|
||||
const String & metadata_file_path_,
|
||||
bool create)
|
||||
: remote_fs_root_path(remote_fs_root_path_)
|
||||
, disk_path(disk_path_)
|
||||
, metadata_file_path(metadata_file_path_)
|
||||
, total_size(0), remote_fs_objects(0), ref_count(0)
|
||||
{
|
||||
if (create)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */
|
||||
|
||||
UInt32 version;
|
||||
readIntText(version, buf);
|
||||
|
||||
if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}",
|
||||
disk_path + metadata_file_path, toString(version), toString(VERSION_READ_ONLY_FLAG));
|
||||
|
||||
assertChar('\n', buf);
|
||||
|
||||
UInt32 remote_fs_objects_count;
|
||||
readIntText(remote_fs_objects_count, buf);
|
||||
assertChar('\t', buf);
|
||||
readIntText(total_size, buf);
|
||||
assertChar('\n', buf);
|
||||
remote_fs_objects.resize(remote_fs_objects_count);
|
||||
|
||||
for (size_t i = 0; i < remote_fs_objects_count; ++i)
|
||||
{
|
||||
String remote_fs_object_path;
|
||||
size_t remote_fs_object_size;
|
||||
readIntText(remote_fs_object_size, buf);
|
||||
assertChar('\t', buf);
|
||||
readEscapedString(remote_fs_object_path, buf);
|
||||
if (version == VERSION_ABSOLUTE_PATHS)
|
||||
{
|
||||
if (!boost::algorithm::starts_with(remote_fs_object_path, remote_fs_root_path))
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Path in metadata does not correspond S3 root path. Path: {}, root path: {}, disk path: {}",
|
||||
remote_fs_object_path, remote_fs_root_path, disk_path_);
|
||||
|
||||
remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size());
|
||||
}
|
||||
assertChar('\n', buf);
|
||||
remote_fs_objects[i] = {remote_fs_object_path, remote_fs_object_size};
|
||||
}
|
||||
|
||||
readIntText(ref_count, buf);
|
||||
assertChar('\n', buf);
|
||||
|
||||
if (version >= VERSION_READ_ONLY_FLAG)
|
||||
{
|
||||
readBoolText(read_only, buf);
|
||||
assertChar('\n', buf);
|
||||
}
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
throw;
|
||||
|
||||
throw Exception("Failed to read metadata file", e, ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
}
|
||||
|
||||
void IDiskRemote::Metadata::addObject(const String & path, size_t size)
|
||||
{
|
||||
total_size += size;
|
||||
remote_fs_objects.emplace_back(path, size);
|
||||
}
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void IDiskRemote::Metadata::save(bool sync)
|
||||
{
|
||||
WriteBufferFromFile buf(disk_path + metadata_file_path, 1024);
|
||||
|
||||
writeIntText(VERSION_RELATIVE_PATHS, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeIntText(remote_fs_objects.size(), buf);
|
||||
writeChar('\t', buf);
|
||||
writeIntText(total_size, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : remote_fs_objects)
|
||||
{
|
||||
writeIntText(remote_fs_object_size, buf);
|
||||
writeChar('\t', buf);
|
||||
writeEscapedString(remote_fs_object_path, buf);
|
||||
writeChar('\n', buf);
|
||||
}
|
||||
|
||||
writeIntText(ref_count, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeBoolText(read_only, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
buf.finalize();
|
||||
if (sync)
|
||||
buf.sync();
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readOrCreateMetaForWriting(const String & path, WriteMode mode)
|
||||
{
|
||||
bool exist = exists(path);
|
||||
if (exist)
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
if (metadata.read_only)
|
||||
throw Exception("File is read-only: " + path, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
|
||||
if (mode == WriteMode::Rewrite)
|
||||
removeFile(path); /// Remove for re-write.
|
||||
else
|
||||
return metadata;
|
||||
}
|
||||
|
||||
auto metadata = createMeta(path);
|
||||
/// Save empty metadata to disk to have ability to get file size while buffer is not finalized.
|
||||
metadata.save();
|
||||
|
||||
return metadata;
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readMeta(const String & path) const
|
||||
{
|
||||
return Metadata(remote_fs_root_path, metadata_path, path);
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::createMeta(const String & path) const
|
||||
{
|
||||
return Metadata(remote_fs_root_path, metadata_path, path, true);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper)
|
||||
{
|
||||
LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path));
|
||||
|
||||
Poco::File file(metadata_path + path);
|
||||
|
||||
if (!file.isFile())
|
||||
throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path);
|
||||
|
||||
try
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
|
||||
/// If there is no references - delete content from remote FS.
|
||||
if (metadata.ref_count == 0)
|
||||
{
|
||||
file.remove();
|
||||
for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects)
|
||||
fs_paths_keeper->addPath(remote_fs_root_path + remote_fs_object_path);
|
||||
}
|
||||
else /// In other case decrement number of references, save metadata and delete file.
|
||||
{
|
||||
--metadata.ref_count;
|
||||
metadata.save();
|
||||
file.remove();
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// If it's impossible to read meta - just remove it from FS.
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
{
|
||||
LOG_WARNING(log,
|
||||
"Metadata file {} can't be read by reason: {}. Removing it forcibly.",
|
||||
backQuote(path), e.nested() ? e.nested()->message() : e.message());
|
||||
file.remove();
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
Poco::File file(metadata_path + path);
|
||||
if (file.isFile())
|
||||
{
|
||||
removeMeta(path, fs_paths_keeper);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it{iterateDirectory(path)}; it->isValid(); it->next())
|
||||
removeMetaRecursive(it->path(), fs_paths_keeper);
|
||||
file.remove();
|
||||
}
|
||||
}
|
||||
|
||||
DiskPtr DiskRemoteReservation::getDisk(size_t i) const
|
||||
{
|
||||
if (i != 0)
|
||||
throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX);
|
||||
return disk;
|
||||
}
|
||||
|
||||
|
||||
void DiskRemoteReservation::update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
disk->reserved_bytes -= size;
|
||||
size = new_size;
|
||||
disk->reserved_bytes += size;
|
||||
}
|
||||
|
||||
|
||||
DiskRemoteReservation::~DiskRemoteReservation()
|
||||
{
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(disk->log, "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->reserved_bytes -= size;
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(disk->log, "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::IDiskRemote(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & metadata_path_,
|
||||
const String & log_name_,
|
||||
size_t thread_pool_size)
|
||||
: IDisk(std::make_unique<AsyncExecutor>(log_name_, thread_pool_size))
|
||||
, log(&Poco::Logger::get(log_name_))
|
||||
, name(name_)
|
||||
, remote_fs_root_path(remote_fs_root_path_)
|
||||
, metadata_path(metadata_path_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::exists(const String & path) const
|
||||
{
|
||||
return Poco::File(metadata_path + path).exists();
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::isFile(const String & path) const
|
||||
{
|
||||
return Poco::File(metadata_path + path).isFile();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createFile(const String & path)
|
||||
{
|
||||
/// Create empty metadata file.
|
||||
auto metadata = createMeta(path);
|
||||
metadata.save();
|
||||
}
|
||||
|
||||
|
||||
size_t IDiskRemote::getFileSize(const String & path) const
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
return metadata.total_size;
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::moveFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
|
||||
Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::replaceFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
{
|
||||
const String tmp_path = to_path + ".old";
|
||||
moveFile(to_path, tmp_path);
|
||||
moveFile(from_path, to_path);
|
||||
removeFile(tmp_path);
|
||||
}
|
||||
else
|
||||
moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeFileIfExists(const String & path)
|
||||
{
|
||||
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
|
||||
if (Poco::File(metadata_path + path).exists())
|
||||
{
|
||||
removeMeta(path, fs_paths_keeper);
|
||||
removeFromRemoteFS(fs_paths_keeper);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeSharedFile(const String & path, bool keep_in_remote_fs)
|
||||
{
|
||||
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
|
||||
removeMeta(path, fs_paths_keeper);
|
||||
if (!keep_in_remote_fs)
|
||||
removeFromRemoteFS(fs_paths_keeper);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeSharedRecursive(const String & path, bool keep_in_remote_fs)
|
||||
{
|
||||
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
|
||||
removeMetaRecursive(path, fs_paths_keeper);
|
||||
if (!keep_in_remote_fs)
|
||||
removeFromRemoteFS(fs_paths_keeper);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::setReadOnly(const String & path)
|
||||
{
|
||||
/// We should store read only flag inside metadata file (instead of using FS flag),
|
||||
/// because we modify metadata file when create hard-links from it.
|
||||
auto metadata = readMeta(path);
|
||||
metadata.read_only = true;
|
||||
metadata.save();
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::isDirectory(const String & path) const
|
||||
{
|
||||
return Poco::File(metadata_path + path).isDirectory();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createDirectory(const String & path)
|
||||
{
|
||||
Poco::File(metadata_path + path).createDirectory();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createDirectories(const String & path)
|
||||
{
|
||||
Poco::File(metadata_path + path).createDirectories();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::clearDirectory(const String & path)
|
||||
{
|
||||
for (auto it{iterateDirectory(path)}; it->isValid(); it->next())
|
||||
if (isFile(it->path()))
|
||||
removeFile(it->path());
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeDirectory(const String & path)
|
||||
{
|
||||
Poco::File(metadata_path + path).remove();
|
||||
}
|
||||
|
||||
|
||||
DiskDirectoryIteratorPtr IDiskRemote::iterateDirectory(const String & path)
|
||||
{
|
||||
return std::make_unique<RemoteDiskDirectoryIterator>(metadata_path + path, path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::listFiles(const String & path, std::vector<String> & file_names)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
file_names.push_back(it->name());
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp)
|
||||
{
|
||||
Poco::File(metadata_path + path).setLastModified(timestamp);
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp IDiskRemote::getLastModified(const String & path)
|
||||
{
|
||||
return Poco::File(metadata_path + path).getLastModified();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createHardLink(const String & src_path, const String & dst_path)
|
||||
{
|
||||
/// Increment number of references.
|
||||
auto src = readMeta(src_path);
|
||||
++src.ref_count;
|
||||
src.save();
|
||||
|
||||
/// Create FS hardlink to metadata file.
|
||||
DB::createHardLink(metadata_path + src_path, metadata_path + dst_path);
|
||||
}
|
||||
|
||||
|
||||
ReservationPtr IDiskRemote::reserve(UInt64 bytes)
|
||||
{
|
||||
if (!tryReserve(bytes))
|
||||
return {};
|
||||
|
||||
return std::make_unique<DiskRemoteReservation>(std::static_pointer_cast<IDiskRemote>(shared_from_this()), bytes);
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::tryReserve(UInt64 bytes)
|
||||
{
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
|
||||
auto available_space = getAvailableSpace();
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_DEBUG(log, "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
283
src/Disks/IDiskRemote.h
Normal file
283
src/Disks/IDiskRemote.h
Normal file
@ -0,0 +1,283 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <atomic>
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include "Disks/Executor.h"
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <utility>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Helper class to collect paths into chunks of maximum size.
|
||||
/// For s3 it is Aws::vector<ObjectIdentifier>, for hdfs it is std::vector<std::string>.
|
||||
class RemoteFSPathKeeper
|
||||
{
|
||||
public:
|
||||
RemoteFSPathKeeper(size_t chunk_limit_) : chunk_limit(chunk_limit_) {}
|
||||
|
||||
virtual ~RemoteFSPathKeeper() = default;
|
||||
|
||||
virtual void addPath(const String & path) = 0;
|
||||
|
||||
protected:
|
||||
size_t chunk_limit;
|
||||
};
|
||||
|
||||
using RemoteFSPathKeeperPtr = std::shared_ptr<RemoteFSPathKeeper>;
|
||||
|
||||
|
||||
/// Base Disk class for remote FS's, which are not posix-compatible (DiskS3 and DiskHDFS)
|
||||
class IDiskRemote : public IDisk
|
||||
{
|
||||
|
||||
friend class DiskRemoteReservation;
|
||||
|
||||
public:
|
||||
IDiskRemote(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & metadata_path_,
|
||||
const String & log_name_,
|
||||
size_t thread_pool_size);
|
||||
|
||||
struct Metadata;
|
||||
|
||||
const String & getName() const final override { return name; }
|
||||
|
||||
const String & getPath() const final override { return metadata_path; }
|
||||
|
||||
Metadata readMeta(const String & path) const;
|
||||
|
||||
Metadata createMeta(const String & path) const;
|
||||
|
||||
Metadata readOrCreateMetaForWriting(const String & path, WriteMode mode);
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
|
||||
void removeFileIfExists(const String & path) override;
|
||||
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false); }
|
||||
|
||||
void removeSharedFile(const String & path, bool keep_in_remote_fs) override;
|
||||
|
||||
void removeSharedRecursive(const String & path, bool keep_in_remote_fs) override;
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
void removeDirectory(const String & path) override;
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
virtual void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) = 0;
|
||||
|
||||
virtual RemoteFSPathKeeperPtr createFSPathKeeper() const = 0;
|
||||
|
||||
protected:
|
||||
Poco::Logger * log;
|
||||
const String name;
|
||||
const String remote_fs_root_path;
|
||||
|
||||
const String metadata_path;
|
||||
|
||||
private:
|
||||
void removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper);
|
||||
|
||||
void removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper);
|
||||
|
||||
bool tryReserve(UInt64 bytes);
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
};
|
||||
|
||||
using RemoteDiskPtr = std::shared_ptr<IDiskRemote>;
|
||||
|
||||
/// Remote FS (S3, HDFS) metadata file layout:
|
||||
/// Number of FS objects, Total size of all FS objects.
|
||||
/// Each FS object represents path where object located in FS and size of object.
|
||||
|
||||
struct IDiskRemote::Metadata
|
||||
{
|
||||
/// Metadata file version.
|
||||
static constexpr UInt32 VERSION_ABSOLUTE_PATHS = 1;
|
||||
static constexpr UInt32 VERSION_RELATIVE_PATHS = 2;
|
||||
static constexpr UInt32 VERSION_READ_ONLY_FLAG = 3;
|
||||
|
||||
using PathAndSize = std::pair<String, size_t>;
|
||||
|
||||
/// Remote FS (S3, HDFS) root path.
|
||||
const String & remote_fs_root_path;
|
||||
|
||||
/// Disk path.
|
||||
const String & disk_path;
|
||||
|
||||
/// Relative path to metadata file on local FS.
|
||||
String metadata_file_path;
|
||||
|
||||
/// Total size of all remote FS (S3, HDFS) objects.
|
||||
size_t total_size = 0;
|
||||
|
||||
/// Remote FS (S3, HDFS) objects paths and their sizes.
|
||||
std::vector<PathAndSize> remote_fs_objects;
|
||||
|
||||
/// Number of references (hardlinks) to this metadata file.
|
||||
UInt32 ref_count = 0;
|
||||
|
||||
/// Flag indicates that file is read only.
|
||||
bool read_only = false;
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
Metadata(const String & remote_fs_root_path_,
|
||||
const String & disk_path_,
|
||||
const String & metadata_file_path_,
|
||||
bool create = false);
|
||||
|
||||
void addObject(const String & path, size_t size);
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void save(bool sync = false);
|
||||
|
||||
};
|
||||
|
||||
|
||||
class RemoteDiskDirectoryIterator final : public IDiskDirectoryIterator
|
||||
{
|
||||
public:
|
||||
RemoteDiskDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {}
|
||||
|
||||
void next() override { ++iter; }
|
||||
|
||||
bool isValid() const override { return iter != Poco::DirectoryIterator(); }
|
||||
|
||||
String path() const override
|
||||
{
|
||||
if (iter->isDirectory())
|
||||
return folder_path + iter.name() + '/';
|
||||
else
|
||||
return folder_path + iter.name();
|
||||
}
|
||||
|
||||
String name() const override { return iter.name(); }
|
||||
|
||||
private:
|
||||
Poco::DirectoryIterator iter;
|
||||
String folder_path;
|
||||
};
|
||||
|
||||
|
||||
class DiskRemoteReservation final : public IReservation
|
||||
{
|
||||
public:
|
||||
DiskRemoteReservation(const RemoteDiskPtr & disk_, UInt64 size_)
|
||||
: disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
|
||||
{
|
||||
}
|
||||
|
||||
UInt64 getSize() const override { return size; }
|
||||
|
||||
DiskPtr getDisk(size_t i) const override;
|
||||
|
||||
Disks getDisks() const override { return {disk}; }
|
||||
|
||||
void update(UInt64 new_size) override;
|
||||
|
||||
~DiskRemoteReservation() override;
|
||||
|
||||
private:
|
||||
RemoteDiskPtr disk;
|
||||
UInt64 size;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
|
||||
/// Runs tasks asynchronously using thread pool.
|
||||
class AsyncExecutor : public Executor
|
||||
{
|
||||
public:
|
||||
explicit AsyncExecutor(const String & name_, int thread_pool_size)
|
||||
: name(name_)
|
||||
, pool(ThreadPool(thread_pool_size)) {}
|
||||
|
||||
std::future<void> execute(std::function<void()> task) override
|
||||
{
|
||||
auto promise = std::make_shared<std::promise<void>>();
|
||||
pool.scheduleOrThrowOnError(
|
||||
[promise, task]()
|
||||
{
|
||||
try
|
||||
{
|
||||
task();
|
||||
promise->set_value();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Failed to run async task");
|
||||
|
||||
try
|
||||
{
|
||||
promise->set_exception(std::current_exception());
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
});
|
||||
|
||||
return promise->get_future();
|
||||
}
|
||||
|
||||
void setMaxThreads(size_t threads)
|
||||
{
|
||||
pool.setMaxThreads(threads);
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
ThreadPool pool;
|
||||
};
|
||||
|
||||
}
|
128
src/Disks/ReadIndirectBufferFromRemoteFS.cpp
Normal file
128
src/Disks/ReadIndirectBufferFromRemoteFS.cpp
Normal file
@ -0,0 +1,128 @@
|
||||
#include "ReadIndirectBufferFromRemoteFS.h"
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
ReadIndirectBufferFromRemoteFS<T>::ReadIndirectBufferFromRemoteFS(
|
||||
IDiskRemote::Metadata metadata_)
|
||||
: metadata(std::move(metadata_))
|
||||
{
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
off_t ReadIndirectBufferFromRemoteFS<T>::seek(off_t offset_, int whence)
|
||||
{
|
||||
if (whence == SEEK_CUR)
|
||||
{
|
||||
/// If position within current working buffer - shift pos.
|
||||
if (!working_buffer.empty() && size_t(getPosition() + offset_) < absolute_position)
|
||||
{
|
||||
pos += offset_;
|
||||
return getPosition();
|
||||
}
|
||||
else
|
||||
{
|
||||
absolute_position += offset_;
|
||||
}
|
||||
}
|
||||
else if (whence == SEEK_SET)
|
||||
{
|
||||
/// If position within current working buffer - shift pos.
|
||||
if (!working_buffer.empty() && size_t(offset_) >= absolute_position - working_buffer.size()
|
||||
&& size_t(offset_) < absolute_position)
|
||||
{
|
||||
pos = working_buffer.end() - (absolute_position - offset_);
|
||||
return getPosition();
|
||||
}
|
||||
else
|
||||
{
|
||||
absolute_position = offset_;
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
current_buf = initialize();
|
||||
pos = working_buffer.end();
|
||||
|
||||
return absolute_position;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
std::unique_ptr<T> ReadIndirectBufferFromRemoteFS<T>::initialize()
|
||||
{
|
||||
size_t offset = absolute_position;
|
||||
for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i)
|
||||
{
|
||||
current_buf_idx = i;
|
||||
const auto & [file_path, size] = metadata.remote_fs_objects[i];
|
||||
if (size > offset)
|
||||
{
|
||||
auto buf = createReadBuffer(file_path);
|
||||
buf->seek(offset, SEEK_SET);
|
||||
return buf;
|
||||
}
|
||||
offset -= size;
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
bool ReadIndirectBufferFromRemoteFS<T>::nextImpl()
|
||||
{
|
||||
/// Find first available buffer that fits to given offset.
|
||||
if (!current_buf)
|
||||
current_buf = initialize();
|
||||
|
||||
/// If current buffer has remaining data - use it.
|
||||
if (current_buf && current_buf->next())
|
||||
{
|
||||
working_buffer = current_buf->buffer();
|
||||
absolute_position += working_buffer.size();
|
||||
return true;
|
||||
}
|
||||
|
||||
/// If there is no available buffers - nothing to read.
|
||||
if (current_buf_idx + 1 >= metadata.remote_fs_objects.size())
|
||||
return false;
|
||||
|
||||
++current_buf_idx;
|
||||
const auto & path = metadata.remote_fs_objects[current_buf_idx].first;
|
||||
|
||||
current_buf = createReadBuffer(path);
|
||||
current_buf->next();
|
||||
|
||||
working_buffer = current_buf->buffer();
|
||||
absolute_position += working_buffer.size();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
#if USE_AWS_S3
|
||||
template
|
||||
class ReadIndirectBufferFromRemoteFS<ReadBufferFromS3>;
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
template
|
||||
class ReadIndirectBufferFromRemoteFS<ReadBufferFromHDFS>;
|
||||
#endif
|
||||
|
||||
}
|
||||
|
||||
#endif
|
46
src/Disks/ReadIndirectBufferFromRemoteFS.h
Normal file
46
src/Disks/ReadIndirectBufferFromRemoteFS.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <utility>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Reads data from S3/HDFS using stored paths in metadata.
|
||||
template <typename T>
|
||||
class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
ReadIndirectBufferFromRemoteFS(IDiskRemote::Metadata metadata_);
|
||||
|
||||
off_t seek(off_t offset_, int whence) override;
|
||||
|
||||
off_t getPosition() override { return absolute_position - available(); }
|
||||
|
||||
String getFileName() const override { return metadata.metadata_file_path; }
|
||||
|
||||
virtual std::unique_ptr<T> createReadBuffer(const String & path) = 0;
|
||||
|
||||
protected:
|
||||
IDiskRemote::Metadata metadata;
|
||||
|
||||
private:
|
||||
std::unique_ptr<T> initialize();
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
size_t absolute_position = 0;
|
||||
|
||||
size_t current_buf_idx = 0;
|
||||
|
||||
std::unique_ptr<T> current_buf;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
File diff suppressed because it is too large
Load Diff
@ -2,7 +2,6 @@
|
||||
|
||||
#include <atomic>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include "Disks/Executor.h"
|
||||
|
||||
@ -12,6 +11,7 @@
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <re2/re2.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -28,7 +28,8 @@ struct DiskS3Settings
|
||||
size_t min_bytes_for_seek_,
|
||||
bool send_metadata_,
|
||||
int thread_pool_size_,
|
||||
int list_object_keys_size_);
|
||||
int list_object_keys_size_,
|
||||
int objects_chunk_size_to_delete_);
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
size_t s3_max_single_read_retries;
|
||||
@ -38,25 +39,24 @@ struct DiskS3Settings
|
||||
bool send_metadata;
|
||||
int thread_pool_size;
|
||||
int list_object_keys_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in S3 and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains S3 object key with actual data.
|
||||
*/
|
||||
class DiskS3 : public IDisk
|
||||
class DiskS3 final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using ObjectMetadata = std::map<std::string, std::string>;
|
||||
using Futures = std::vector<std::future<void>>;
|
||||
|
||||
using SettingsPtr = std::unique_ptr<DiskS3Settings>;
|
||||
using GetDiskSettings = std::function<SettingsPtr(const Poco::Util::AbstractConfiguration &, const String, ContextConstPtr)>;
|
||||
|
||||
friend class DiskS3Reservation;
|
||||
|
||||
class AwsS3KeyKeeper;
|
||||
struct Metadata;
|
||||
struct RestoreInformation;
|
||||
|
||||
DiskS3(
|
||||
@ -67,44 +67,6 @@ public:
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_);
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
|
||||
const String & getPath() const override { return metadata_path; }
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
void moveFile(const String & from_path, const String & to_path, bool send_metadata);
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
@ -118,25 +80,16 @@ public:
|
||||
size_t buf_size,
|
||||
WriteMode mode) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
void removeFileIfExists(const String & path) override;
|
||||
void removeDirectory(const String & path) override;
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false); }
|
||||
void removeFromRemoteFS(RemoteFSPathKeeperPtr keeper) override;
|
||||
|
||||
void removeSharedFile(const String & path, bool keep_s3) override;
|
||||
void removeSharedRecursive(const String & path, bool keep_s3) override;
|
||||
RemoteFSPathKeeperPtr createFSPathKeeper() const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path, bool send_metadata);
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
void createHardLink(const String & src_path, const String & dst_path, bool send_metadata);
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
DiskType::Type getType() const override { return DiskType::Type::S3; }
|
||||
|
||||
void shutdown() override;
|
||||
@ -157,16 +110,6 @@ public:
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextConstPtr context) override;
|
||||
|
||||
private:
|
||||
bool tryReserve(UInt64 bytes);
|
||||
|
||||
void removeMeta(const String & path, AwsS3KeyKeeper & keys);
|
||||
void removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys);
|
||||
void removeAws(const AwsS3KeyKeeper & keys);
|
||||
|
||||
Metadata readOrCreateMetaForWriting(const String & path, WriteMode mode);
|
||||
Metadata readMeta(const String & path) const;
|
||||
Metadata createMeta(const String & path) const;
|
||||
|
||||
void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata);
|
||||
/// Converts revision to binary string with leading zeroes (64 bit).
|
||||
static String revisionToString(UInt64 revision);
|
||||
@ -200,19 +143,12 @@ private:
|
||||
/// Forms detached path '../../detached/part_name/' from '../../part_name/'
|
||||
static String pathToDetached(const String & source_path);
|
||||
|
||||
const String name;
|
||||
const String bucket;
|
||||
const String s3_root_path;
|
||||
const String metadata_path;
|
||||
MultiVersion<DiskS3Settings> current_settings;
|
||||
|
||||
MultiVersion<DiskS3Settings> current_settings;
|
||||
/// Gets disk settings from context.
|
||||
GetDiskSettings settings_getter;
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
|
||||
std::atomic<UInt64> revision_counter = 0;
|
||||
static constexpr UInt64 LATEST_REVISION = std::numeric_limits<UInt64>::max();
|
||||
static constexpr UInt64 UNKNOWN_REVISION = 0;
|
||||
@ -229,8 +165,6 @@ private:
|
||||
static constexpr int RESTORABLE_SCHEMA_VERSION = 1;
|
||||
/// Directories with data.
|
||||
const std::vector<String> data_roots {"data", "store"};
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("DiskS3");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -156,7 +156,8 @@ std::unique_ptr<DiskS3Settings> getSettings(const Poco::Util::AbstractConfigurat
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getBool(config_prefix + ".send_metadata", false),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".list_object_keys_size", 1000));
|
||||
config.getInt(config_prefix + ".list_object_keys_size", 1000),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000));
|
||||
}
|
||||
|
||||
}
|
||||
@ -225,4 +226,3 @@ void registerDiskS3(DiskFactory & factory)
|
||||
void registerDiskS3(DiskFactory &) {}
|
||||
|
||||
#endif
|
||||
|
||||
|
71
src/Disks/WriteIndirectBufferFromRemoteFS.cpp
Normal file
71
src/Disks/WriteIndirectBufferFromRemoteFS.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include "WriteIndirectBufferFromRemoteFS.h"
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS.
|
||||
template <typename T>
|
||||
WriteIndirectBufferFromRemoteFS<T>::WriteIndirectBufferFromRemoteFS(
|
||||
std::unique_ptr<T> impl_,
|
||||
IDiskRemote::Metadata metadata_,
|
||||
const String & remote_fs_path_)
|
||||
: WriteBufferFromFileDecorator(std::move(impl_))
|
||||
, metadata(std::move(metadata_))
|
||||
, remote_fs_path(remote_fs_path_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
WriteIndirectBufferFromRemoteFS<T>::~WriteIndirectBufferFromRemoteFS()
|
||||
{
|
||||
try
|
||||
{
|
||||
WriteIndirectBufferFromRemoteFS::finalize();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void WriteIndirectBufferFromRemoteFS<T>::finalize()
|
||||
{
|
||||
if (finalized)
|
||||
return;
|
||||
|
||||
WriteBufferFromFileDecorator::finalize();
|
||||
|
||||
metadata.addObject(remote_fs_path, count());
|
||||
metadata.save();
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void WriteIndirectBufferFromRemoteFS<T>::sync()
|
||||
{
|
||||
if (finalized)
|
||||
metadata.save(true);
|
||||
}
|
||||
|
||||
|
||||
#if USE_AWS_S3
|
||||
template
|
||||
class WriteIndirectBufferFromRemoteFS<WriteBufferFromS3>;
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
template
|
||||
class WriteIndirectBufferFromRemoteFS<WriteBufferFromHDFS>;
|
||||
#endif
|
||||
|
||||
}
|
||||
|
||||
#endif
|
39
src/Disks/WriteIndirectBufferFromRemoteFS.h
Normal file
39
src/Disks/WriteIndirectBufferFromRemoteFS.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFileDecorator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS.
|
||||
template <typename T>
|
||||
class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator
|
||||
{
|
||||
public:
|
||||
WriteIndirectBufferFromRemoteFS(
|
||||
std::unique_ptr<T> impl_,
|
||||
IDiskRemote::Metadata metadata_,
|
||||
const String & remote_fs_path_);
|
||||
|
||||
virtual ~WriteIndirectBufferFromRemoteFS() override;
|
||||
|
||||
void finalize() override;
|
||||
|
||||
void sync() override;
|
||||
|
||||
String getFileName() const override { return metadata.metadata_file_path; }
|
||||
|
||||
private:
|
||||
IDiskRemote::Metadata metadata;
|
||||
|
||||
String remote_fs_path;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -8,21 +8,33 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDiskLocal(DiskFactory & factory);
|
||||
void registerDiskMemory(DiskFactory & factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
void registerDiskS3(DiskFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
void registerDiskHDFS(DiskFactory & factory);
|
||||
#endif
|
||||
|
||||
|
||||
void registerDisks()
|
||||
{
|
||||
auto & factory = DiskFactory::instance();
|
||||
|
||||
registerDiskLocal(factory);
|
||||
registerDiskMemory(factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
registerDiskS3(factory);
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
registerDiskHDFS(factory);
|
||||
#endif
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,14 +1,15 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "gtest_disk.h"
|
||||
|
||||
|
||||
#if !defined(__clang__)
|
||||
# pragma GCC diagnostic push
|
||||
# pragma GCC diagnostic ignored "-Wsuggest-override"
|
||||
#endif
|
||||
|
||||
|
||||
template <typename T>
|
||||
DB::DiskPtr createDisk();
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/DiskMemory.h>
|
||||
#include <Disks/IDisk.h>
|
||||
|
160
src/Disks/tests/gtest_disk_hdfs.cpp
Normal file
160
src/Disks/tests/gtest_disk_hdfs.cpp
Normal file
@ -0,0 +1,160 @@
|
||||
#include <gtest/gtest.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "gtest_disk.h"
|
||||
|
||||
|
||||
#define RUN_HDFS_TEST 0
|
||||
#if RUN_HDFS_TEST
|
||||
|
||||
#include <Disks/HDFS/DiskHDFS.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
const String hdfs_uri = "hdfs://172.20.0.2:9000/disk_test/";
|
||||
const String metadata_path = "/path/to/metadata/";
|
||||
const String config_path = "/path/to/config.xml";
|
||||
const String file_name = "test.txt";
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, RemoveFileHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
DB::HDFSBuilderWrapper builder = DB::createHDFSBuilder(hdfs_uri, *config);
|
||||
DB::HDFSFSPtr fs = DB::createHDFSFS(builder.get());
|
||||
|
||||
disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
auto metadata = disk.readMeta(file_name);
|
||||
|
||||
const String hdfs_file_name = metadata.remote_fs_objects[0].first;
|
||||
const String hdfs_file_path = "/disk_test/" + hdfs_file_name;
|
||||
|
||||
auto ret = hdfsExists(fs.get(), hdfs_file_path.data());
|
||||
EXPECT_EQ(0, ret);
|
||||
|
||||
disk.removeFile(file_name);
|
||||
ret = hdfsExists(fs.get(), hdfs_file_path.data());
|
||||
EXPECT_EQ(-1, ret);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, WriteReadHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
{
|
||||
auto out = disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
writeString("Test write to file", *out);
|
||||
}
|
||||
|
||||
{
|
||||
DB::String result;
|
||||
auto in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("Test write to file", result);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, RewriteFileHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
for (size_t i = 1; i <= 10; ++i)
|
||||
{
|
||||
std::unique_ptr<DB::WriteBuffer> out = disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
writeString("Text" + DB::toString(i), *out);
|
||||
}
|
||||
|
||||
{
|
||||
String result;
|
||||
auto in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("Text10", result);
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("", result);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, AppendFileHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
{
|
||||
std::unique_ptr<DB::WriteBuffer> out = disk.writeFile(file_name, 1024, DB::WriteMode::Append);
|
||||
writeString("Text", *out);
|
||||
for (size_t i = 0; i < 10; ++i)
|
||||
{
|
||||
writeIntText(i, *out);
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
String result, expected;
|
||||
auto in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("Text0123456789", result);
|
||||
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("", result);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, SeekHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
{
|
||||
std::unique_ptr<DB::WriteBuffer> out = disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
writeString("test data", *out);
|
||||
}
|
||||
|
||||
/// Test SEEK_SET
|
||||
{
|
||||
String buf(4, '0');
|
||||
std::unique_ptr<DB::SeekableReadBuffer> in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
|
||||
in->seek(5, SEEK_SET);
|
||||
|
||||
in->readStrict(buf.data(), 4);
|
||||
EXPECT_EQ("data", buf);
|
||||
}
|
||||
|
||||
/// Test SEEK_CUR
|
||||
{
|
||||
std::unique_ptr<DB::SeekableReadBuffer> in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
String buf(4, '0');
|
||||
|
||||
in->readStrict(buf.data(), 4);
|
||||
EXPECT_EQ("test", buf);
|
||||
|
||||
// Skip whitespace
|
||||
in->seek(1, SEEK_CUR);
|
||||
|
||||
in->readStrict(buf.data(), 4);
|
||||
EXPECT_EQ("data", buf);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
#endif
|
@ -7,8 +7,7 @@ PEERDIR(
|
||||
)
|
||||
|
||||
|
||||
SRCS(
|
||||
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F examples | grep -v -F S3 | sed 's/^\.\// /' | sort ?>
|
||||
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F examples | grep -v -F 'S3|HDFS' | sed 's/^\.\// /' | sort ?>
|
||||
)
|
||||
|
||||
END()
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <common/types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -18,7 +19,6 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
@ -26,43 +26,6 @@ namespace ErrorCodes
|
||||
* CustomWeek Transformations.
|
||||
*/
|
||||
|
||||
static inline UInt32 dateIsNotSupported(const char * name)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
/// This factor transformation will say that the function is monotone everywhere.
|
||||
struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(Int64, UInt8, const DateLUTImpl &) { return 0; }
|
||||
};
|
||||
|
||||
struct ToWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToYearWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toYearWeek";
|
||||
@ -110,10 +73,34 @@ struct ToStartOfWeekImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
struct ToWeekImpl
|
||||
{
|
||||
explicit Transformer(Transform transform_)
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct WeekTransformer
|
||||
{
|
||||
explicit WeekTransformer(Transform transform_)
|
||||
: transform(std::move(transform_))
|
||||
{}
|
||||
|
||||
@ -139,7 +126,7 @@ struct CustomWeekTransformImpl
|
||||
template <typename Transform>
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, Transform transform = {})
|
||||
{
|
||||
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
|
||||
UInt8 week_mode = DEFAULT_WEEK_MODE;
|
||||
if (arguments.size() > 1)
|
||||
|
@ -172,4 +172,10 @@ struct ModuloImpl
|
||||
#endif
|
||||
};
|
||||
|
||||
template <typename A, typename B>
|
||||
struct ModuloLegacyImpl : ModuloImpl<A, B>
|
||||
{
|
||||
using ResultType = typename NumberTraits::ResultOfModuloLegacy<A, B>::Type;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
@ -143,15 +144,15 @@ public:
|
||||
|
||||
if (checkAndGetDataType<DataTypeDate>(&type))
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
else
|
||||
{
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), DEFAULT_WEEK_MODE, date_lut)
|
||||
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut)
|
||||
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), date_lut)
|
||||
? is_monotonic
|
||||
: is_not_monotonic;
|
||||
}
|
||||
|
@ -96,6 +96,11 @@ struct ModuloByConstantImpl
|
||||
}
|
||||
};
|
||||
|
||||
template <typename A, typename B>
|
||||
struct ModuloLegacyByConstantImpl : ModuloByConstantImpl<A, B>
|
||||
{
|
||||
using Op = ModuloLegacyImpl<A, B>;
|
||||
};
|
||||
}
|
||||
|
||||
/** Specializations are specified for dividing numbers of the type UInt64 and UInt32 by the numbers of the same sign.
|
||||
@ -134,4 +139,12 @@ void registerFunctionModulo(FunctionFactory & factory)
|
||||
factory.registerAlias("mod", "modulo", FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
struct NameModuloLegacy { static constexpr auto name = "moduloLegacy"; };
|
||||
using FunctionModuloLegacy = BinaryArithmeticOverloadResolver<ModuloLegacyImpl, NameModuloLegacy, false>;
|
||||
|
||||
void registerFunctionModuloLegacy(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionModuloLegacy>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -11,6 +11,7 @@ void registerFunctionIntDiv(FunctionFactory & factory);
|
||||
void registerFunctionIntDivOrZero(FunctionFactory & factory);
|
||||
void registerFunctionModulo(FunctionFactory & factory);
|
||||
void registerFunctionModuloOrZero(FunctionFactory & factory);
|
||||
void registerFunctionModuloLegacy(FunctionFactory & factory);
|
||||
void registerFunctionNegate(FunctionFactory & factory);
|
||||
void registerFunctionAbs(FunctionFactory & factory);
|
||||
void registerFunctionBitAnd(FunctionFactory & factory);
|
||||
@ -51,6 +52,7 @@ void registerFunctionsArithmetic(FunctionFactory & factory)
|
||||
registerFunctionIntDivOrZero(factory);
|
||||
registerFunctionModulo(factory);
|
||||
registerFunctionModuloOrZero(factory);
|
||||
registerFunctionModuloLegacy(factory);
|
||||
registerFunctionNegate(factory);
|
||||
registerFunctionAbs(factory);
|
||||
registerFunctionBitAnd(factory);
|
||||
|
@ -463,12 +463,6 @@ struct ContextSharedPart
|
||||
dictionaries_xmls.reset();
|
||||
|
||||
delete_system_logs = std::move(system_logs);
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
if (auto * cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||
cache->reset();
|
||||
#endif
|
||||
|
||||
embedded_dictionaries.reset();
|
||||
external_dictionaries_loader.reset();
|
||||
models_repository_guard.reset();
|
||||
|
@ -28,7 +28,6 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
static CHJIT & getJITInstance()
|
||||
@ -43,13 +42,36 @@ static Poco::Logger * getLogger()
|
||||
return &logger;
|
||||
}
|
||||
|
||||
class CompiledFunction
|
||||
{
|
||||
public:
|
||||
|
||||
CompiledFunction(void * compiled_function_, CHJIT::CompiledModuleInfo module_info_)
|
||||
: compiled_function(compiled_function_)
|
||||
, module_info(std::move(module_info_))
|
||||
{}
|
||||
|
||||
void * getCompiledFunction() const { return compiled_function; }
|
||||
|
||||
~CompiledFunction()
|
||||
{
|
||||
getJITInstance().deleteCompiledModule(module_info);
|
||||
}
|
||||
|
||||
private:
|
||||
|
||||
void * compiled_function;
|
||||
|
||||
CHJIT::CompiledModuleInfo module_info;
|
||||
};
|
||||
|
||||
class LLVMExecutableFunction : public IExecutableFunction
|
||||
{
|
||||
public:
|
||||
|
||||
explicit LLVMExecutableFunction(const std::string & name_, JITCompiledFunction function_)
|
||||
explicit LLVMExecutableFunction(const std::string & name_, std::shared_ptr<CompiledFunction> compiled_function_)
|
||||
: name(name_)
|
||||
, function(function_)
|
||||
, compiled_function(compiled_function_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -81,7 +103,9 @@ public:
|
||||
}
|
||||
|
||||
columns[arguments.size()] = getColumnData(result_column.get());
|
||||
function(input_rows_count, columns.data());
|
||||
|
||||
JITCompiledFunction jit_compiled_function_typed = reinterpret_cast<JITCompiledFunction>(compiled_function->getCompiledFunction());
|
||||
jit_compiled_function_typed(input_rows_count, columns.data());
|
||||
|
||||
#if defined(MEMORY_SANITIZER)
|
||||
/// Memory sanitizer don't know about stores from JIT-ed code.
|
||||
@ -111,7 +135,7 @@ public:
|
||||
|
||||
private:
|
||||
std::string name;
|
||||
JITCompiledFunction function = nullptr;
|
||||
std::shared_ptr<CompiledFunction> compiled_function;
|
||||
};
|
||||
|
||||
class LLVMFunction : public IFunctionBase
|
||||
@ -131,17 +155,13 @@ public:
|
||||
else if (node.type == CompileDAG::CompileType::INPUT)
|
||||
argument_types.emplace_back(node.result_type);
|
||||
}
|
||||
|
||||
module_info = compileFunction(getJITInstance(), *this);
|
||||
}
|
||||
|
||||
~LLVMFunction() override
|
||||
void setCompiledFunction(std::shared_ptr<CompiledFunction> compiled_function_)
|
||||
{
|
||||
getJITInstance().deleteCompiledModule(module_info);
|
||||
compiled_function = compiled_function_;
|
||||
}
|
||||
|
||||
size_t getCompiledSize() const { return module_info.size; }
|
||||
|
||||
bool isCompilable() const override { return true; }
|
||||
|
||||
llvm::Value * compile(llvm::IRBuilderBase & builder, Values values) const override
|
||||
@ -157,13 +177,10 @@ public:
|
||||
|
||||
ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override
|
||||
{
|
||||
void * function = getJITInstance().findCompiledFunction(module_info, name);
|
||||
if (!compiled_function)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Compiled function was not initialized {}", name);
|
||||
|
||||
if (!function)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot find compiled function {}", name);
|
||||
|
||||
JITCompiledFunction function_typed = reinterpret_cast<JITCompiledFunction>(function);
|
||||
return std::make_unique<LLVMExecutableFunction>(name, function_typed);
|
||||
return std::make_unique<LLVMExecutableFunction>(name, compiled_function);
|
||||
}
|
||||
|
||||
bool isDeterministic() const override
|
||||
@ -252,7 +269,7 @@ private:
|
||||
CompileDAG dag;
|
||||
DataTypes argument_types;
|
||||
std::vector<FunctionBasePtr> nested_functions;
|
||||
CHJIT::CompiledModuleInfo module_info;
|
||||
std::shared_ptr<CompiledFunction> compiled_function;
|
||||
};
|
||||
|
||||
static FunctionBasePtr compile(
|
||||
@ -271,43 +288,42 @@ static FunctionBasePtr compile(
|
||||
|
||||
LOG_TRACE(getLogger(), "Try to compile expression {}", dag.dump());
|
||||
|
||||
FunctionBasePtr fn;
|
||||
auto llvm_function = std::make_shared<LLVMFunction>(dag);
|
||||
|
||||
if (auto * compilation_cache = CompiledExpressionCacheFactory::instance().tryGetCache())
|
||||
{
|
||||
auto [compiled_function, was_inserted] = compilation_cache->getOrSet(hash_key, [&dag] ()
|
||||
|
||||
auto [compiled_function_cache_entry, was_inserted] = compilation_cache->getOrSet(hash_key, [&] ()
|
||||
{
|
||||
auto llvm_function = std::make_unique<LLVMFunction>(dag);
|
||||
size_t compiled_size = llvm_function->getCompiledSize();
|
||||
CHJIT::CompiledModuleInfo compiled_module_info = compileFunction(getJITInstance(), *llvm_function);
|
||||
auto * compiled_jit_function = getJITInstance().findCompiledFunction(compiled_module_info, llvm_function->getName());
|
||||
auto compiled_function = std::make_shared<CompiledFunction>(compiled_jit_function, compiled_module_info);
|
||||
|
||||
CompiledFunction function
|
||||
{
|
||||
.function = std::move(llvm_function),
|
||||
.compiled_size = compiled_size
|
||||
};
|
||||
|
||||
return std::make_shared<CompiledFunction>(function);
|
||||
return std::make_shared<CompiledFunctionCacheEntry>(std::move(compiled_function), compiled_module_info.size);
|
||||
});
|
||||
|
||||
if (was_inserted)
|
||||
LOG_TRACE(getLogger(),
|
||||
"Put compiled expression {} in cache used cache size {} total cache size {}",
|
||||
compiled_function->function->getName(),
|
||||
llvm_function->getName(),
|
||||
compilation_cache->weight(),
|
||||
compilation_cache->maxSize());
|
||||
else
|
||||
LOG_TRACE(getLogger(), "Get compiled expression {} from cache", compiled_function->function->getName());
|
||||
LOG_TRACE(getLogger(), "Get compiled expression {} from cache", llvm_function->getName());
|
||||
|
||||
fn = compiled_function->function;
|
||||
llvm_function->setCompiledFunction(compiled_function_cache_entry->getCompiledFunction());
|
||||
}
|
||||
else
|
||||
{
|
||||
fn = std::make_unique<LLVMFunction>(dag);
|
||||
CHJIT::CompiledModuleInfo compiled_module_info = compileFunction(getJITInstance(), *llvm_function);
|
||||
auto * compiled_jit_function = getJITInstance().findCompiledFunction(compiled_module_info, llvm_function->getName());
|
||||
auto compiled_function = std::make_shared<CompiledFunction>(compiled_jit_function, compiled_module_info);
|
||||
llvm_function->setCompiledFunction(compiled_function);
|
||||
}
|
||||
|
||||
LOG_TRACE(getLogger(), "Use compiled expression {}", fn->getName());
|
||||
LOG_TRACE(getLogger(), "Use compiled expression {}", llvm_function->getName());
|
||||
|
||||
return fn;
|
||||
return llvm_function;
|
||||
}
|
||||
|
||||
static bool isCompilableConstant(const ActionsDAG::Node & node)
|
||||
|
@ -5,35 +5,47 @@
|
||||
#endif
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
# include <Functions/IFunction.h>
|
||||
# include <Common/LRUCache.h>
|
||||
# include <Common/HashTable/Hash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct CompiledFunction
|
||||
class CompiledFunction;
|
||||
|
||||
class CompiledFunctionCacheEntry
|
||||
{
|
||||
FunctionBasePtr function;
|
||||
size_t compiled_size;
|
||||
public:
|
||||
CompiledFunctionCacheEntry(std::shared_ptr<CompiledFunction> compiled_function_, size_t compiled_function_size_)
|
||||
: compiled_function(std::move(compiled_function_))
|
||||
, compiled_function_size(compiled_function_size_)
|
||||
{}
|
||||
|
||||
std::shared_ptr<CompiledFunction> getCompiledFunction() const { return compiled_function; }
|
||||
|
||||
size_t getCompiledFunctionSize() const { return compiled_function_size; }
|
||||
|
||||
private:
|
||||
std::shared_ptr<CompiledFunction> compiled_function;
|
||||
|
||||
size_t compiled_function_size;
|
||||
};
|
||||
|
||||
struct CompiledFunctionWeightFunction
|
||||
{
|
||||
size_t operator()(const CompiledFunction & compiled_function) const
|
||||
size_t operator()(const CompiledFunctionCacheEntry & compiled_function) const
|
||||
{
|
||||
return compiled_function.compiled_size;
|
||||
return compiled_function.getCompiledFunctionSize();
|
||||
}
|
||||
};
|
||||
|
||||
/** This child of LRUCache breaks one of it's invariants: total weight may be changed after insertion.
|
||||
* We have to do so, because we don't known real memory consumption of generated LLVM code for every function.
|
||||
*/
|
||||
class CompiledExpressionCache : public LRUCache<UInt128, CompiledFunction, UInt128Hash, CompiledFunctionWeightFunction>
|
||||
class CompiledExpressionCache : public LRUCache<UInt128, CompiledFunctionCacheEntry, UInt128Hash, CompiledFunctionWeightFunction>
|
||||
{
|
||||
public:
|
||||
using Base = LRUCache<UInt128, CompiledFunction, UInt128Hash, CompiledFunctionWeightFunction>;
|
||||
using Base = LRUCache<UInt128, CompiledFunctionCacheEntry, UInt128Hash, CompiledFunctionWeightFunction>;
|
||||
using Base::Base;
|
||||
};
|
||||
|
||||
|
@ -1,57 +1,57 @@
|
||||
#include <iostream>
|
||||
|
||||
// #include <llvm/IR/IRBuilder.h>
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
|
||||
// #include <Interpreters/JIT/CHJIT.h>
|
||||
#include <Interpreters/JIT/CHJIT.h>
|
||||
|
||||
// void test_function()
|
||||
// {
|
||||
// std::cerr << "Test function" << std::endl;
|
||||
// }
|
||||
void test_function()
|
||||
{
|
||||
std::cerr << "Test function" << std::endl;
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
(void)(argc);
|
||||
(void)(argv);
|
||||
|
||||
// auto jit = DB::CHJIT();
|
||||
auto jit = DB::CHJIT();
|
||||
|
||||
// jit.registerExternalSymbol("test_function", reinterpret_cast<void *>(&test_function));
|
||||
jit.registerExternalSymbol("test_function", reinterpret_cast<void *>(&test_function));
|
||||
|
||||
// auto compiled_module_info = jit.compileModule([](llvm::Module & module)
|
||||
// {
|
||||
// auto & context = module.getContext();
|
||||
// llvm::IRBuilder<> b (context);
|
||||
auto compiled_module_info = jit.compileModule([](llvm::Module & module)
|
||||
{
|
||||
auto & context = module.getContext();
|
||||
llvm::IRBuilder<> b (context);
|
||||
|
||||
// auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||
// auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||
auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||
auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||
|
||||
// auto * value_type = b.getInt64Ty();
|
||||
// auto * pointer_type = value_type->getPointerTo();
|
||||
auto * value_type = b.getInt64Ty();
|
||||
auto * pointer_type = value_type->getPointerTo();
|
||||
|
||||
// auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||
// auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||
// auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||
auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||
auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||
|
||||
// auto * argument = function->args().begin();
|
||||
// b.SetInsertPoint(entry);
|
||||
auto * argument = function->args().begin();
|
||||
b.SetInsertPoint(entry);
|
||||
|
||||
// b.CreateCall(func_declaration);
|
||||
b.CreateCall(func_declaration);
|
||||
|
||||
// auto * load_argument = b.CreateLoad(value_type, argument);
|
||||
// auto * value = b.CreateAdd(load_argument, load_argument);
|
||||
// b.CreateRet(value);
|
||||
// });
|
||||
auto * load_argument = b.CreateLoad(value_type, argument);
|
||||
auto * value = b.CreateAdd(load_argument, load_argument);
|
||||
b.CreateRet(value);
|
||||
});
|
||||
|
||||
// for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||
// {
|
||||
// std::cerr << compiled_function_name << std::endl;
|
||||
// }
|
||||
for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||
{
|
||||
std::cerr << compiled_function_name << std::endl;
|
||||
}
|
||||
|
||||
// int64_t value = 5;
|
||||
// auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction(compiled_module_info, "test_name"));
|
||||
// auto result = test_name_function(&value);
|
||||
// std::cerr << "Result " << result << std::endl;
|
||||
int64_t value = 5;
|
||||
auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction(compiled_module_info, "test_name"));
|
||||
auto result = test_name_function(&value);
|
||||
std::cerr << "Result " << result << std::endl;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
@ -145,6 +145,7 @@ HDFSBuilderWrapper createHDFSBuilder(const String & uri_str, const Poco::Util::A
|
||||
|
||||
hdfsBuilderSetUserName(builder.get(), user.c_str());
|
||||
}
|
||||
|
||||
hdfsBuilderSetNameNode(builder.get(), host.c_str());
|
||||
if (port != 0)
|
||||
{
|
||||
|
@ -1,13 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
#include <memory>
|
||||
#include <type_traits>
|
||||
#include <vector>
|
||||
|
||||
#include <hdfs/hdfs.h>
|
||||
#include <hdfs/hdfs.h> // Y_IGNORE
|
||||
#include <common/types.h>
|
||||
#include <mutex>
|
||||
|
||||
|
@ -13,6 +13,8 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int NETWORK_ERROR;
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
extern const int SEEK_POSITION_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
ReadBufferFromHDFS::~ReadBufferFromHDFS() = default;
|
||||
@ -29,6 +31,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
|
||||
HDFSBuilderWrapper builder;
|
||||
HDFSFSPtr fs;
|
||||
|
||||
off_t offset = 0;
|
||||
bool initialized = false;
|
||||
|
||||
explicit ReadBufferFromHDFSImpl(
|
||||
const std::string & hdfs_uri_,
|
||||
const std::string & hdfs_file_path_,
|
||||
@ -48,8 +53,30 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
|
||||
hdfs_uri + hdfs_file_path, std::string(hdfsGetLastError()));
|
||||
}
|
||||
|
||||
int read(char * start, size_t size) const
|
||||
~ReadBufferFromHDFSImpl()
|
||||
{
|
||||
std::lock_guard lock(hdfs_init_mutex);
|
||||
hdfsCloseFile(fs.get(), fin);
|
||||
}
|
||||
|
||||
void initialize() const
|
||||
{
|
||||
if (!offset)
|
||||
return;
|
||||
|
||||
int seek_status = hdfsSeek(fs.get(), fin, offset);
|
||||
if (seek_status != 0)
|
||||
throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Fail to seek HDFS file: {}, error: {}", hdfs_uri, std::string(hdfsGetLastError()));
|
||||
}
|
||||
|
||||
int read(char * start, size_t size)
|
||||
{
|
||||
if (!initialized)
|
||||
{
|
||||
initialize();
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
int bytes_read = hdfsRead(fs.get(), fin, start, size);
|
||||
if (bytes_read < 0)
|
||||
throw Exception(ErrorCodes::NETWORK_ERROR,
|
||||
@ -58,10 +85,25 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
|
||||
return bytes_read;
|
||||
}
|
||||
|
||||
~ReadBufferFromHDFSImpl()
|
||||
int seek(off_t offset_, int whence)
|
||||
{
|
||||
std::lock_guard lock(hdfs_init_mutex);
|
||||
hdfsCloseFile(fs.get(), fin);
|
||||
if (initialized)
|
||||
throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
if (whence != SEEK_SET)
|
||||
throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
if (offset_ < 0)
|
||||
throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bounds. Offset: {}", std::to_string(offset_));
|
||||
|
||||
offset = offset_;
|
||||
|
||||
return offset;
|
||||
}
|
||||
|
||||
int tell() const
|
||||
{
|
||||
return offset;
|
||||
}
|
||||
};
|
||||
|
||||
@ -73,7 +115,7 @@ ReadBufferFromHDFS::ReadBufferFromHDFS(
|
||||
const String & hdfs_file_path_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
size_t buf_size_)
|
||||
: BufferWithOwnMemory<ReadBuffer>(buf_size_)
|
||||
: BufferWithOwnMemory<SeekableReadBuffer>(buf_size_)
|
||||
, impl(std::make_unique<ReadBufferFromHDFSImpl>(hdfs_uri_, hdfs_file_path_, config_))
|
||||
{
|
||||
}
|
||||
@ -90,6 +132,18 @@ bool ReadBufferFromHDFS::nextImpl()
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
off_t ReadBufferFromHDFS::seek(off_t off, int whence)
|
||||
{
|
||||
return impl->seek(off, whence);
|
||||
}
|
||||
|
||||
|
||||
off_t ReadBufferFromHDFS::getPosition()
|
||||
{
|
||||
return impl->tell() + count();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -7,28 +7,34 @@
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <string>
|
||||
#include <memory>
|
||||
#include <hdfs/hdfs.h>
|
||||
#include <hdfs/hdfs.h> // Y_IGNORE
|
||||
#include <common/types.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Accepts HDFS path to file and opens it.
|
||||
* Closes file by himself (thus "owns" a file descriptor).
|
||||
*/
|
||||
class ReadBufferFromHDFS : public BufferWithOwnMemory<ReadBuffer>
|
||||
class ReadBufferFromHDFS : public BufferWithOwnMemory<SeekableReadBuffer>
|
||||
{
|
||||
struct ReadBufferFromHDFSImpl;
|
||||
|
||||
public:
|
||||
ReadBufferFromHDFS(const String & hdfs_uri_, const String & hdfs_file_path_,
|
||||
const Poco::Util::AbstractConfiguration &, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
const Poco::Util::AbstractConfiguration & config_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
|
||||
~ReadBufferFromHDFS() override;
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
off_t seek(off_t offset_, int whence) override;
|
||||
|
||||
off_t getPosition() override;
|
||||
|
||||
private:
|
||||
std::unique_ptr<ReadBufferFromHDFSImpl> impl;
|
||||
};
|
||||
|
@ -64,7 +64,6 @@ public:
|
||||
struct SourcesInfo
|
||||
{
|
||||
std::vector<String> uris;
|
||||
|
||||
std::atomic<size_t> next_uri_to_read = 0;
|
||||
|
||||
bool need_path_column = false;
|
||||
|
@ -27,20 +27,24 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
|
||||
HDFSBuilderWrapper builder;
|
||||
HDFSFSPtr fs;
|
||||
|
||||
explicit WriteBufferFromHDFSImpl(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_)
|
||||
: hdfs_uri(hdfs_name_)
|
||||
, builder(createHDFSBuilder(hdfs_uri,config_))
|
||||
explicit WriteBufferFromHDFSImpl(
|
||||
const std::string & hdfs_uri_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
int flags)
|
||||
: hdfs_uri(hdfs_uri_)
|
||||
, builder(createHDFSBuilder(hdfs_uri, config_))
|
||||
, fs(createHDFSFS(builder.get()))
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const String path = hdfs_uri.substr(begin_of_path);
|
||||
|
||||
if (path.find_first_of("*?{") != std::string::npos)
|
||||
throw Exception("URI '" + hdfs_uri + "' contains globs, so the table is in readonly mode", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "URI '{}' contains globs, so the table is in readonly mode", hdfs_uri);
|
||||
|
||||
if (!hdfsExists(fs.get(), path.c_str()))
|
||||
throw Exception("File: " + path + " is already exists", ErrorCodes::BAD_ARGUMENTS);
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} already exists", path);
|
||||
|
||||
fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here
|
||||
fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here
|
||||
|
||||
if (fout == nullptr)
|
||||
{
|
||||
@ -76,9 +80,13 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
|
||||
}
|
||||
};
|
||||
|
||||
WriteBufferFromHDFS::WriteBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, size_t buf_size_)
|
||||
WriteBufferFromHDFS::WriteBufferFromHDFS(
|
||||
const std::string & hdfs_name_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
size_t buf_size_,
|
||||
int flags_)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buf_size_)
|
||||
, impl(std::make_unique<WriteBufferFromHDFSImpl>(hdfs_name_, config_))
|
||||
, impl(std::make_unique<WriteBufferFromHDFSImpl>(hdfs_name_, config_, flags_))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Accepts HDFS path to file and opens it.
|
||||
@ -15,11 +16,13 @@ namespace DB
|
||||
*/
|
||||
class WriteBufferFromHDFS final : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
struct WriteBufferFromHDFSImpl;
|
||||
std::unique_ptr<WriteBufferFromHDFSImpl> impl;
|
||||
|
||||
public:
|
||||
WriteBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration &, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
WriteBufferFromHDFS(
|
||||
const std::string & hdfs_name_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
int flags = O_WRONLY);
|
||||
|
||||
WriteBufferFromHDFS(WriteBufferFromHDFS &&) = default;
|
||||
|
||||
@ -30,6 +33,11 @@ public:
|
||||
void sync() override;
|
||||
|
||||
void finalize() override;
|
||||
|
||||
private:
|
||||
struct WriteBufferFromHDFSImpl;
|
||||
std::unique_ptr<WriteBufferFromHDFSImpl> impl;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/TreeRewriter.h>
|
||||
@ -86,6 +87,30 @@ KeyDescription KeyDescription::getKeyFromAST(
|
||||
return getSortingKeyFromAST(definition_ast, columns, context, {});
|
||||
}
|
||||
|
||||
bool KeyDescription::moduloToModuloLegacyRecursive(ASTPtr node_expr)
|
||||
{
|
||||
if (!node_expr)
|
||||
return false;
|
||||
|
||||
auto * function_expr = node_expr->as<ASTFunction>();
|
||||
bool modulo_in_ast = false;
|
||||
if (function_expr)
|
||||
{
|
||||
if (function_expr->name == "modulo")
|
||||
{
|
||||
function_expr->name = "moduloLegacy";
|
||||
modulo_in_ast = true;
|
||||
}
|
||||
if (function_expr->arguments)
|
||||
{
|
||||
auto children = function_expr->arguments->children;
|
||||
for (const auto & child : children)
|
||||
modulo_in_ast |= moduloToModuloLegacyRecursive(child);
|
||||
}
|
||||
}
|
||||
return modulo_in_ast;
|
||||
}
|
||||
|
||||
KeyDescription KeyDescription::getSortingKeyFromAST(
|
||||
const ASTPtr & definition_ast,
|
||||
const ColumnsDescription & columns,
|
||||
|
@ -69,6 +69,9 @@ struct KeyDescription
|
||||
/// unintentionaly share AST variables and modify them.
|
||||
KeyDescription(const KeyDescription & other);
|
||||
KeyDescription & operator=(const KeyDescription & other);
|
||||
|
||||
/// Substitute modulo with moduloLegacy. Used in KeyCondition to allow proper comparison with keys.
|
||||
static bool moduloToModuloLegacyRecursive(ASTPtr node_expr);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -379,6 +379,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name)
|
||||
|
||||
MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr context,
|
||||
const String & part_name,
|
||||
const String & replica_path,
|
||||
const String & host,
|
||||
@ -470,9 +471,36 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
size_t sum_files_size = 0;
|
||||
readBinary(sum_files_size, in);
|
||||
IMergeTreeDataPart::TTLInfos ttl_infos;
|
||||
/// Skip ttl infos, not required for S3 metadata
|
||||
String ttl_infos_string;
|
||||
readBinary(ttl_infos_string, in);
|
||||
ReadBufferFromString ttl_infos_buffer(ttl_infos_string);
|
||||
assertString("ttl format version: 1\n", ttl_infos_buffer);
|
||||
ttl_infos.read(ttl_infos_buffer);
|
||||
|
||||
ReservationPtr reservation
|
||||
= data.balancedReservation(metadata_snapshot, sum_files_size, 0, part_name, part_info, {}, tagger_ptr, &ttl_infos, true);
|
||||
if (!reservation)
|
||||
reservation
|
||||
= data.reserveSpacePreferringTTLRules(metadata_snapshot, sum_files_size, ttl_infos, std::time(nullptr), 0, true);
|
||||
if (reservation)
|
||||
{
|
||||
/// When we have multi-volume storage, one of them was chosen, depends on TTL, free space, etc.
|
||||
/// Chosen one may be S3 or not.
|
||||
DiskPtr disk = reservation->getDisk();
|
||||
if (disk && disk->getType() == DiskType::Type::S3)
|
||||
{
|
||||
for (const auto & d : disks_s3)
|
||||
{
|
||||
if (d->getPath() == disk->getPath())
|
||||
{
|
||||
Disks disks_tmp = { disk };
|
||||
disks_s3.swap(disks_tmp);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String part_type = "Wide";
|
||||
readStringBinary(part_type, in);
|
||||
if (part_type == "InMemory")
|
||||
@ -493,7 +521,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
if (e.code() != ErrorCodes::S3_ERROR)
|
||||
throw;
|
||||
/// Try again but without S3 copy
|
||||
return fetchPart(metadata_snapshot, part_name, replica_path, host, port, timeouts,
|
||||
return fetchPart(metadata_snapshot, context, part_name, replica_path, host, port, timeouts,
|
||||
user, password, interserver_scheme, to_detached, tmp_prefix_, nullptr, false);
|
||||
}
|
||||
}
|
||||
@ -557,7 +585,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
|
||||
|
||||
MergeTreeData::DataPart::Checksums checksums;
|
||||
return part_type == "InMemory"
|
||||
? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, std::move(reservation), in, projections)
|
||||
? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, context, std::move(reservation), in, projections)
|
||||
: downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, sync, reservation->getDisk(), in, projections, checksums);
|
||||
}
|
||||
|
||||
@ -565,6 +593,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
const String & part_name,
|
||||
const UUID & part_uuid,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr context,
|
||||
ReservationPtr reservation,
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
size_t projections)
|
||||
@ -619,7 +648,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory(
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->setColumns(block.getNamesAndTypesList());
|
||||
new_data_part->minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||
new_data_part->partition.create(metadata_snapshot, block, 0);
|
||||
new_data_part->partition.create(metadata_snapshot, block, 0, context);
|
||||
|
||||
MergedBlockOutputStream part_out(
|
||||
new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
|
||||
@ -795,7 +824,6 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3(
|
||||
readBinary(files, in);
|
||||
|
||||
auto volume = std::make_shared<SingleDiskVolume>("volume_" + part_name, disk);
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path);
|
||||
|
||||
for (size_t i = 0; i < files; ++i)
|
||||
{
|
||||
@ -805,7 +833,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3(
|
||||
readStringBinary(file_name, in);
|
||||
readBinary(file_size, in);
|
||||
|
||||
String data_path = new_data_part->getFullRelativePath() + file_name;
|
||||
String data_path = part_download_path + file_name;
|
||||
String metadata_file = fullPath(disk, data_path);
|
||||
|
||||
{
|
||||
@ -837,6 +865,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToS3(
|
||||
|
||||
assertEOF(in);
|
||||
|
||||
MergeTreeData::MutableDataPartPtr new_data_part = data.createPart(part_name, volume, part_relative_path);
|
||||
new_data_part->is_temp = true;
|
||||
new_data_part->modification_time = time(nullptr);
|
||||
new_data_part->loadColumnsChecksumsIndexes(true, false);
|
||||
|
@ -65,6 +65,7 @@ public:
|
||||
/// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory.
|
||||
MergeTreeData::MutableDataPartPtr fetchPart(
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr context,
|
||||
const String & part_name,
|
||||
const String & replica_path,
|
||||
const String & host,
|
||||
@ -106,6 +107,7 @@ private:
|
||||
const String & part_name,
|
||||
const UUID & part_uuid,
|
||||
const StorageMetadataPtr & metadata_snapshot,
|
||||
ContextPtr context,
|
||||
ReservationPtr reservation,
|
||||
PooledReadWriteBufferFromHTTP & in,
|
||||
size_t projections);
|
||||
|
@ -429,9 +429,9 @@ void IMergeTreeDataPart::removeIfNeeded()
|
||||
}
|
||||
|
||||
if (parent_part)
|
||||
projectionRemove(parent_part->getFullRelativePath());
|
||||
projectionRemove(parent_part->getFullRelativePath(), keep_s3_on_delete);
|
||||
else
|
||||
remove(false);
|
||||
remove(keep_s3_on_delete);
|
||||
|
||||
if (state == State::DeleteOnDestroy)
|
||||
{
|
||||
@ -1108,7 +1108,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
|
||||
if (isProjectionPart())
|
||||
{
|
||||
LOG_WARNING(storage.log, "Projection part {} should be removed by its parent {}.", name, parent_part->name);
|
||||
projectionRemove(parent_part->getFullRelativePath());
|
||||
projectionRemove(parent_part->getFullRelativePath(), keep_s3);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -1158,7 +1158,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
|
||||
std::unordered_set<String> projection_directories;
|
||||
for (const auto & [p_name, projection_part] : projection_parts)
|
||||
{
|
||||
projection_part->projectionRemove(to);
|
||||
projection_part->projectionRemove(to, keep_s3);
|
||||
projection_directories.emplace(p_name + ".proj");
|
||||
}
|
||||
|
||||
@ -1207,7 +1207,7 @@ void IMergeTreeDataPart::remove(bool keep_s3) const
|
||||
}
|
||||
|
||||
|
||||
void IMergeTreeDataPart::projectionRemove(const String & parent_to) const
|
||||
void IMergeTreeDataPart::projectionRemove(const String & parent_to, bool keep_s3) const
|
||||
{
|
||||
String to = parent_to + "/" + relative_path;
|
||||
auto disk = volume->getDisk();
|
||||
@ -1219,7 +1219,7 @@ void IMergeTreeDataPart::projectionRemove(const String & parent_to) const
|
||||
"Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: checksums.txt is missing",
|
||||
fullPath(disk, to));
|
||||
/// If the part is not completely written, we cannot use fast path by listing files.
|
||||
disk->removeRecursive(to + "/");
|
||||
disk->removeSharedRecursive(to + "/", keep_s3);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1232,17 +1232,17 @@ void IMergeTreeDataPart::projectionRemove(const String & parent_to) const
|
||||
# pragma GCC diagnostic ignored "-Wunused-variable"
|
||||
#endif
|
||||
for (const auto & [file, _] : checksums.files)
|
||||
disk->removeFile(to + "/" + file);
|
||||
disk->removeSharedFile(to + "/" + file, keep_s3);
|
||||
#if !defined(__clang__)
|
||||
# pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
for (const auto & file : {"checksums.txt", "columns.txt"})
|
||||
disk->removeFile(to + "/" + file);
|
||||
disk->removeFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME);
|
||||
disk->removeFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME);
|
||||
disk->removeSharedFile(to + "/" + file, keep_s3);
|
||||
disk->removeSharedFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME, keep_s3);
|
||||
disk->removeSharedFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME, keep_s3);
|
||||
|
||||
disk->removeDirectory(to);
|
||||
disk->removeSharedRecursive(to, keep_s3);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -1250,7 +1250,7 @@ void IMergeTreeDataPart::projectionRemove(const String & parent_to) const
|
||||
|
||||
LOG_ERROR(storage.log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(disk, to), getCurrentExceptionMessage(false));
|
||||
|
||||
disk->removeRecursive(to + "/");
|
||||
disk->removeSharedRecursive(to + "/", keep_s3);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -130,7 +130,7 @@ public:
|
||||
|
||||
void remove(bool keep_s3 = false) const;
|
||||
|
||||
void projectionRemove(const String & parent_to) const;
|
||||
void projectionRemove(const String & parent_to, bool keep_s3 = false) const;
|
||||
|
||||
/// Initialize columns (from columns.txt if exists, or create from column files if not).
|
||||
/// Load checksums from checksums.txt if exists. Load index if required.
|
||||
@ -199,18 +199,21 @@ public:
|
||||
/// Frozen by ALTER TABLE ... FREEZE ... It is used for information purposes in system.parts table.
|
||||
mutable std::atomic<bool> is_frozen {false};
|
||||
|
||||
/// Flag for keep S3 data when zero-copy replication over S3 turned on.
|
||||
mutable bool keep_s3_on_delete = false;
|
||||
|
||||
/**
|
||||
* Part state is a stage of its lifetime. States are ordered and state of a part could be increased only.
|
||||
* Part state should be modified under data_parts mutex.
|
||||
*
|
||||
* Possible state transitions:
|
||||
* Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set
|
||||
* Precommitted -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part)
|
||||
* Precommitted -> Committed: we successfully committed a part to active dataset
|
||||
* Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION
|
||||
* Outdated -> Deleting: a cleaner selected this part for deletion
|
||||
* Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion
|
||||
* Committed -> DeleteOnDestroy if part was moved to another disk
|
||||
* Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set
|
||||
* Precommitted -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part)
|
||||
* Precommitted -> Committed: we successfully committed a part to active dataset
|
||||
* Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION
|
||||
* Outdated -> Deleting: a cleaner selected this part for deletion
|
||||
* Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion
|
||||
* Committed -> DeleteOnDestroy: if part was moved to another disk
|
||||
*/
|
||||
enum class State
|
||||
{
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Storages/KeyDescription.h>
|
||||
|
||||
#include <cassert>
|
||||
#include <stack>
|
||||
@ -591,6 +592,30 @@ void KeyCondition::traverseAST(const ASTPtr & node, ContextPtr context, Block &
|
||||
rpn.emplace_back(std::move(element));
|
||||
}
|
||||
|
||||
bool KeyCondition::canConstantBeWrapped(const ASTPtr & node, const String & expr_name, String & result_expr_name)
|
||||
{
|
||||
const auto & sample_block = key_expr->getSampleBlock();
|
||||
|
||||
/// sample_block from key_expr cannot contain modulo and moduloLegacy at the same time.
|
||||
/// For partition key it is always moduloLegacy.
|
||||
if (sample_block.has(expr_name))
|
||||
{
|
||||
result_expr_name = expr_name;
|
||||
}
|
||||
else
|
||||
{
|
||||
auto adjusted_ast = node->clone();
|
||||
KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast);
|
||||
String adjusted_expr_name = adjusted_ast->getColumnName();
|
||||
|
||||
if (!sample_block.has(adjusted_expr_name))
|
||||
return false;
|
||||
|
||||
result_expr_name = adjusted_expr_name;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
|
||||
const ASTPtr & node,
|
||||
@ -600,11 +625,13 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions(
|
||||
DataTypePtr & out_type)
|
||||
{
|
||||
// Constant expr should use alias names if any
|
||||
String expr_name = node->getColumnName();
|
||||
const auto & sample_block = key_expr->getSampleBlock();
|
||||
if (!sample_block.has(expr_name))
|
||||
String passed_expr_name = node->getColumnName();
|
||||
String expr_name;
|
||||
if (!canConstantBeWrapped(node, passed_expr_name, expr_name))
|
||||
return false;
|
||||
|
||||
const auto & sample_block = key_expr->getSampleBlock();
|
||||
|
||||
/// TODO Nullable index is not yet landed.
|
||||
if (out_value.isNull())
|
||||
return false;
|
||||
@ -668,11 +695,13 @@ bool KeyCondition::canConstantBeWrappedByFunctions(
|
||||
const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type)
|
||||
{
|
||||
// Constant expr should use alias names if any
|
||||
String expr_name = ast->getColumnName();
|
||||
const auto & sample_block = key_expr->getSampleBlock();
|
||||
if (!sample_block.has(expr_name))
|
||||
String passed_expr_name = ast->getColumnName();
|
||||
String expr_name;
|
||||
if (!canConstantBeWrapped(ast, passed_expr_name, expr_name))
|
||||
return false;
|
||||
|
||||
const auto & sample_block = key_expr->getSampleBlock();
|
||||
|
||||
/// TODO Nullable index is not yet landed.
|
||||
if (out_value.isNull())
|
||||
return false;
|
||||
|
@ -419,6 +419,12 @@ private:
|
||||
bool canConstantBeWrappedByFunctions(
|
||||
const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type);
|
||||
|
||||
/// Check if ASTPtr node, passed to canConstantBeWrappedBy*, can be used by them for further checks.
|
||||
/// Always call this method at start of other methods, which require key comparison, because it also checks if adjusted
|
||||
/// key expression can also be used (with substitution from modulo to moduloLegacy). This is needed because partition key
|
||||
/// is always modified, when passed into keyCondition, - with recursive substitution from modulo to moduloLegacy.
|
||||
bool canConstantBeWrapped(const ASTPtr & node, const String & expr_name, String & result_expr_name);
|
||||
|
||||
/// If it's possible to make an RPNElement
|
||||
/// that will filter values (possibly tuples) by the content of 'prepared_set',
|
||||
/// do it and return true.
|
||||
|
@ -23,7 +23,7 @@ void MergeTreeBlockOutputStream::writePrefix()
|
||||
|
||||
void MergeTreeBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot);
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
|
||||
for (auto & current_block : part_blocks)
|
||||
{
|
||||
Stopwatch watch;
|
||||
|
@ -773,7 +773,7 @@ std::optional<UInt64> MergeTreeData::totalRowsByPartitionPredicateImpl(
|
||||
// Generate valid expressions for filtering
|
||||
bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, local_context, virtual_columns_block, expression_ast);
|
||||
|
||||
PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, local_context, true /* strict */);
|
||||
PartitionPruner partition_pruner(metadata_snapshot, query_info, local_context, true /* strict */);
|
||||
if (partition_pruner.isUseless() && !valid)
|
||||
return {};
|
||||
|
||||
@ -877,13 +877,13 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks)
|
||||
{
|
||||
/// Create and correctly initialize global WAL object
|
||||
write_ahead_log = std::make_shared<MergeTreeWriteAheadLog>(*this, disk_ptr, it->name());
|
||||
for (auto && part : write_ahead_log->restore(metadata_snapshot))
|
||||
for (auto && part : write_ahead_log->restore(metadata_snapshot, getContext()))
|
||||
parts_from_wal.push_back(std::move(part));
|
||||
}
|
||||
else if (settings->in_memory_parts_enable_wal)
|
||||
{
|
||||
MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name());
|
||||
for (auto && part : wal.restore(metadata_snapshot))
|
||||
for (auto && part : wal.restore(metadata_snapshot, getContext()))
|
||||
parts_from_wal.push_back(std::move(part));
|
||||
}
|
||||
}
|
||||
@ -2726,6 +2726,22 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy)
|
||||
if (active_part_it == data_parts_by_info.end())
|
||||
throw Exception("Cannot swap part '" + part_copy->name + "', no such active part.", ErrorCodes::NO_SUCH_DATA_PART);
|
||||
|
||||
/// We do not check allow_s3_zero_copy_replication here because data may be shared
|
||||
/// when allow_s3_zero_copy_replication turned on and off again
|
||||
|
||||
original_active_part->keep_s3_on_delete = false;
|
||||
|
||||
if (original_active_part->volume->getDisk()->getType() == DiskType::Type::S3)
|
||||
{
|
||||
if (part_copy->volume->getDisk()->getType() == DiskType::Type::S3
|
||||
&& original_active_part->getUniqueId() == part_copy->getUniqueId())
|
||||
{ /// May be when several volumes use the same S3 storage
|
||||
original_active_part->keep_s3_on_delete = true;
|
||||
}
|
||||
else
|
||||
original_active_part->keep_s3_on_delete = !unlockSharedData(*original_active_part);
|
||||
}
|
||||
|
||||
modifyPartState(original_active_part, DataPartState::DeleteOnDestroy);
|
||||
data_parts_indexes.erase(active_part_it);
|
||||
|
||||
|
@ -503,7 +503,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts(
|
||||
|
||||
minmax_idx_condition.emplace(
|
||||
query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)));
|
||||
partition_pruner.emplace(metadata_snapshot_base->getPartitionKey(), query_info, context, false /* strict */);
|
||||
partition_pruner.emplace(metadata_snapshot_base, query_info, context, false /* strict */);
|
||||
|
||||
if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless()))
|
||||
{
|
||||
|
@ -140,7 +140,8 @@ void updateTTL(
|
||||
|
||||
}
|
||||
|
||||
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot)
|
||||
BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(
|
||||
const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
||||
{
|
||||
BlocksWithPartition result;
|
||||
if (!block || !block.rows())
|
||||
@ -155,12 +156,12 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
|
||||
}
|
||||
|
||||
Block block_copy = block;
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
partition_key.expression->execute(block_copy);
|
||||
/// After expression execution partition key columns will be added to block_copy with names regarding partition function.
|
||||
auto partition_key_names_and_types = MergeTreePartition::executePartitionByExpression(metadata_snapshot, block_copy, context);
|
||||
|
||||
ColumnRawPtrs partition_columns;
|
||||
partition_columns.reserve(partition_key.sample_block.columns());
|
||||
for (const ColumnWithTypeAndName & element : partition_key.sample_block)
|
||||
partition_columns.reserve(partition_key_names_and_types.size());
|
||||
for (const auto & element : partition_key_names_and_types)
|
||||
partition_columns.emplace_back(block_copy.getByName(element.name).column.get());
|
||||
|
||||
PODArray<size_t> partition_num_to_first_row;
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
* (split rows by partition)
|
||||
* Works deterministically: if same block was passed, function will return same result in same order.
|
||||
*/
|
||||
static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot);
|
||||
static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
|
||||
|
||||
/** All rows must correspond to same partition.
|
||||
* Returns part with unique name starting with 'tmp_', yet not added to MergeTreeData.
|
||||
|
@ -129,7 +129,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis
|
||||
if (!metadata_snapshot->hasPartitionKey())
|
||||
return;
|
||||
|
||||
const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block;
|
||||
const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block;
|
||||
auto partition_file_path = part_path + "partition.dat";
|
||||
auto file = openForReading(disk, partition_file_path);
|
||||
value.resize(partition_key_sample.columns());
|
||||
@ -140,7 +140,7 @@ void MergeTreePartition::load(const MergeTreeData & storage, const DiskPtr & dis
|
||||
void MergeTreePartition::store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const
|
||||
{
|
||||
auto metadata_snapshot = storage.getInMemoryMetadataPtr();
|
||||
const auto & partition_key_sample = metadata_snapshot->getPartitionKey().sample_block;
|
||||
const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block;
|
||||
store(partition_key_sample, disk, part_path, checksums);
|
||||
}
|
||||
|
||||
@ -153,28 +153,62 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr
|
||||
HashingWriteBuffer out_hashing(*out);
|
||||
for (size_t i = 0; i < value.size(); ++i)
|
||||
partition_key_sample.getByPosition(i).type->getDefaultSerialization()->serializeBinary(value[i], out_hashing);
|
||||
|
||||
out_hashing.next();
|
||||
checksums.files["partition.dat"].file_size = out_hashing.count();
|
||||
checksums.files["partition.dat"].file_hash = out_hashing.getHash();
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row)
|
||||
void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context)
|
||||
{
|
||||
if (!metadata_snapshot->hasPartitionKey())
|
||||
return;
|
||||
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
partition_key.expression->execute(block);
|
||||
size_t partition_columns_num = partition_key.sample_block.columns();
|
||||
value.resize(partition_columns_num);
|
||||
auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block, context);
|
||||
value.resize(partition_key_names_and_types.size());
|
||||
|
||||
for (size_t i = 0; i < partition_columns_num; ++i)
|
||||
/// Executing partition_by expression adds new columns to passed block according to partition functions.
|
||||
/// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back
|
||||
/// with just `modulo`, because it was a temporary substitution.
|
||||
static constexpr auto modulo_legacy_function_name = "moduloLegacy";
|
||||
|
||||
size_t i = 0;
|
||||
for (const auto & element : partition_key_names_and_types)
|
||||
{
|
||||
const auto & column_name = partition_key.sample_block.getByPosition(i).name;
|
||||
const auto & partition_column = block.getByName(column_name).column;
|
||||
partition_column->get(row, value[i]);
|
||||
auto & partition_column = block.getByName(element.name);
|
||||
|
||||
if (element.name.starts_with(modulo_legacy_function_name))
|
||||
partition_column.name = "modulo" + partition_column.name.substr(std::strlen(modulo_legacy_function_name));
|
||||
|
||||
partition_column.column->get(row, value[i++]);
|
||||
}
|
||||
}
|
||||
|
||||
NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context)
|
||||
{
|
||||
auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context);
|
||||
adjusted_partition_key.expression->execute(block);
|
||||
return adjusted_partition_key.sample_block.getNamesAndTypesList();
|
||||
}
|
||||
|
||||
KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
||||
{
|
||||
const auto & partition_key = metadata_snapshot->getPartitionKey();
|
||||
if (!partition_key.definition_ast)
|
||||
return partition_key;
|
||||
|
||||
ASTPtr ast_copy = partition_key.definition_ast->clone();
|
||||
|
||||
/// Implementation of modulo function was changed from 8bit result type to 16bit. For backward compatibility partition by expression is always
|
||||
/// calculated according to previous version - `moduloLegacy`.
|
||||
if (KeyDescription::moduloToModuloLegacyRecursive(ast_copy))
|
||||
{
|
||||
auto adjusted_partition_key = KeyDescription::getKeyFromAST(ast_copy, metadata_snapshot->columns, context);
|
||||
return adjusted_partition_key;
|
||||
}
|
||||
|
||||
return partition_key;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,9 +3,9 @@
|
||||
#include <common/types.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <Storages/KeyDescription.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -41,7 +41,13 @@ public:
|
||||
|
||||
void assign(const MergeTreePartition & other) { value = other.value; }
|
||||
|
||||
void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row);
|
||||
void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context);
|
||||
|
||||
/// Adjust partition key and execute its expression on block. Return sample block according to used expression.
|
||||
static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context);
|
||||
|
||||
/// Make a modified partition key with substitution from modulo to moduloLegacy. Used in paritionPruner.
|
||||
static KeyDescription adjustPartitionKey(const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -111,7 +111,7 @@ void MergeTreeWriteAheadLog::rotate(const std::unique_lock<std::mutex> &)
|
||||
init();
|
||||
}
|
||||
|
||||
MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot)
|
||||
MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
|
||||
{
|
||||
std::unique_lock lock(write_mutex);
|
||||
|
||||
@ -192,7 +192,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
|
||||
MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}));
|
||||
|
||||
part->minmax_idx.update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()));
|
||||
part->partition.create(metadata_snapshot, block, 0);
|
||||
part->partition.create(metadata_snapshot, block, 0, context);
|
||||
if (metadata_snapshot->hasSortingKey())
|
||||
metadata_snapshot->getSortingKey().expression->execute(block);
|
||||
|
||||
|
@ -62,7 +62,7 @@ public:
|
||||
|
||||
void addPart(DataPartInMemoryPtr & part);
|
||||
void dropPart(const String & part_name);
|
||||
std::vector<MergeTreeMutableDataPartPtr> restore(const StorageMetadataPtr & metadata_snapshot);
|
||||
std::vector<MergeTreeMutableDataPartPtr> restore(const StorageMetadataPtr & metadata_snapshot, ContextPtr context);
|
||||
|
||||
using MinMaxBlockNumber = std::pair<Int64, Int64>;
|
||||
static std::optional<MinMaxBlockNumber> tryParseMinMaxBlockNumber(const String & filename);
|
||||
|
@ -14,15 +14,18 @@ class PartitionPruner
|
||||
{
|
||||
private:
|
||||
std::unordered_map<String, bool> partition_filter_map;
|
||||
const KeyDescription & partition_key;
|
||||
|
||||
/// partition_key is adjusted here (with substitution from modulo to moduloLegacy).
|
||||
KeyDescription partition_key;
|
||||
|
||||
KeyCondition partition_condition;
|
||||
bool useless;
|
||||
using DataPart = IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const DataPart>;
|
||||
|
||||
public:
|
||||
PartitionPruner(const KeyDescription & partition_key_, const SelectQueryInfo & query_info, ContextPtr context, bool strict)
|
||||
: partition_key(partition_key_)
|
||||
PartitionPruner(const StorageMetadataPtr & metadata, const SelectQueryInfo & query_info, ContextPtr context, bool strict)
|
||||
: partition_key(MergeTreePartition::adjustPartitionKey(metadata, context))
|
||||
, partition_condition(
|
||||
query_info, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict)
|
||||
, useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue())
|
||||
|
@ -136,7 +136,7 @@ void ReplicatedMergeTreeBlockOutputStream::write(const Block & block)
|
||||
if (quorum)
|
||||
checkQuorumPrecondition(zookeeper);
|
||||
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot);
|
||||
auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context);
|
||||
|
||||
for (auto & current_block : part_blocks)
|
||||
{
|
||||
|
@ -268,9 +268,9 @@ void ReplicatedMergeTreeQueue::removeCoveredPartsFromMutations(const String & pa
|
||||
|
||||
bool some_mutations_are_probably_done = false;
|
||||
|
||||
for (auto it = in_partition->second.begin(); it != in_partition->second.end(); ++it)
|
||||
for (auto & it : in_partition->second)
|
||||
{
|
||||
MutationStatus & status = *it->second;
|
||||
MutationStatus & status = *it.second;
|
||||
|
||||
if (remove_part && remove_covered_parts)
|
||||
status.parts_to_do.removePartAndCoveredParts(part_name);
|
||||
|
@ -2500,7 +2500,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
|
||||
throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
part_desc->res_part = fetcher.fetchPart(
|
||||
metadata_snapshot, part_desc->found_new_part_name, source_replica_path,
|
||||
metadata_snapshot, getContext(), part_desc->found_new_part_name, source_replica_path,
|
||||
address.host, address.replication_port, timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, false, TMP_PREFIX + "fetch_");
|
||||
|
||||
/// TODO: check columns_version of fetched part
|
||||
@ -2616,7 +2616,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return fetcher.fetchPart(
|
||||
metadata_snapshot, entry.new_part_name, source_replica_path,
|
||||
metadata_snapshot, getContext(), entry.new_part_name, source_replica_path,
|
||||
address.host, address.replication_port,
|
||||
timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, true);
|
||||
};
|
||||
@ -4016,6 +4016,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora
|
||||
|
||||
return fetcher.fetchPart(
|
||||
metadata_snapshot,
|
||||
getContext(),
|
||||
part_name,
|
||||
source_replica_path,
|
||||
address.host,
|
||||
@ -4171,7 +4172,7 @@ bool StorageReplicatedMergeTree::fetchExistsPart(const String & part_name, const
|
||||
ErrorCodes::INTERSERVER_SCHEME_DOESNT_MATCH);
|
||||
|
||||
return fetcher.fetchPart(
|
||||
metadata_snapshot, part_name, source_replica_path,
|
||||
metadata_snapshot, getContext(), part_name, source_replica_path,
|
||||
address.host, address.replication_port,
|
||||
timeouts, credentials->getUser(), credentials->getPassword(), interserver_scheme, false, "", nullptr, true,
|
||||
replaced_disk);
|
||||
|
@ -11,6 +11,10 @@
|
||||
<disk_memory>
|
||||
<type>memory</type>
|
||||
</disk_memory>
|
||||
<disk_hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>http://hdfs1:9000/data/</endpoint>
|
||||
</disk_hdfs>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
</yandex>
|
||||
|
@ -5,6 +5,7 @@ disk_types = {
|
||||
"default": "local",
|
||||
"disk_s3": "s3",
|
||||
"disk_memory": "memory",
|
||||
"disk_hdfs": "hdfs",
|
||||
}
|
||||
|
||||
|
||||
@ -12,7 +13,7 @@ disk_types = {
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True)
|
||||
cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True, with_hdfs=True)
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
@ -35,3 +36,4 @@ def test_select_by_type(cluster):
|
||||
node = cluster.instances["node"]
|
||||
for name, disk_type in list(disk_types.items()):
|
||||
assert node.query("SELECT name FROM system.disks WHERE type='" + disk_type + "'") == name + "\n"
|
||||
|
||||
|
0
tests/integration/test_log_family_hdfs/__init__.py
Normal file
0
tests/integration/test_log_family_hdfs/__init__.py
Normal file
@ -0,0 +1,12 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
</yandex>
|
@ -0,0 +1,11 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
|
||||
</hdfs>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
</yandex>
|
59
tests/integration/test_log_family_hdfs/test.py
Normal file
59
tests/integration/test_log_family_hdfs/test.py
Normal file
@ -0,0 +1,59 @@
|
||||
import logging
|
||||
import sys
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster.add_instance("node",
|
||||
main_configs=["configs/storage_conf.xml", "configs/config.d/log_conf.xml"],
|
||||
with_hdfs=True)
|
||||
logging.info("Starting cluster...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
fs.mkdirs('/clickhouse')
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def assert_objects_count(cluster, objects_count, path='data/'):
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert objects_count == len(hdfs_objects)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
"log_engine,files_overhead,files_overhead_per_insert",
|
||||
[("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)])
|
||||
def test_log_family_hdfs(cluster, log_engine, files_overhead, files_overhead_per_insert):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
node.query("CREATE TABLE hdfs_test (id UInt64) ENGINE={} SETTINGS disk = 'hdfs'".format(log_engine))
|
||||
|
||||
node.query("INSERT INTO hdfs_test SELECT number FROM numbers(5)")
|
||||
assert node.query("SELECT * FROM hdfs_test") == "0\n1\n2\n3\n4\n"
|
||||
assert_objects_count(cluster, files_overhead_per_insert + files_overhead)
|
||||
|
||||
node.query("INSERT INTO hdfs_test SELECT number + 5 FROM numbers(3)")
|
||||
assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n"
|
||||
assert_objects_count(cluster, files_overhead_per_insert * 2 + files_overhead)
|
||||
|
||||
node.query("INSERT INTO hdfs_test SELECT number + 8 FROM numbers(1)")
|
||||
assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n8\n"
|
||||
assert_objects_count(cluster, files_overhead_per_insert * 3 + files_overhead)
|
||||
|
||||
node.query("TRUNCATE TABLE hdfs_test")
|
||||
assert_objects_count(cluster, 0)
|
||||
|
||||
node.query("DROP TABLE hdfs_test")
|
||||
|
0
tests/integration/test_merge_tree_hdfs/__init__.py
Normal file
0
tests/integration/test_merge_tree_hdfs/__init__.py
Normal file
@ -0,0 +1,12 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
</yandex>
|
@ -0,0 +1,30 @@
|
||||
<yandex>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
|
||||
</hdfs>
|
||||
<hdd>
|
||||
<type>local</type>
|
||||
<path>/</path>
|
||||
</hdd>
|
||||
</disks>
|
||||
<policies>
|
||||
<hdfs>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>hdfs</disk>
|
||||
</main>
|
||||
<external>
|
||||
<disk>hdd</disk>
|
||||
</external>
|
||||
</volumes>
|
||||
</hdfs>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
</merge_tree>
|
||||
</yandex>
|
317
tests/integration/test_merge_tree_hdfs/test.py
Normal file
317
tests/integration/test_merge_tree_hdfs/test.py
Normal file
@ -0,0 +1,317 @@
|
||||
import logging
|
||||
import random
|
||||
import string
|
||||
import time
|
||||
import threading
|
||||
import os
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.d/storage_conf.xml')
|
||||
|
||||
|
||||
def create_table(cluster, table_name, additional_settings=None):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
create_table_statement = """
|
||||
CREATE TABLE {} (
|
||||
dt Date, id Int64, data String,
|
||||
INDEX min_max (id) TYPE minmax GRANULARITY 3
|
||||
) ENGINE=MergeTree()
|
||||
PARTITION BY dt
|
||||
ORDER BY (dt, id)
|
||||
SETTINGS
|
||||
storage_policy='hdfs',
|
||||
old_parts_lifetime=0,
|
||||
index_granularity=512
|
||||
""".format(table_name)
|
||||
|
||||
if additional_settings:
|
||||
create_table_statement += ","
|
||||
create_table_statement += additional_settings
|
||||
|
||||
node.query(create_table_statement)
|
||||
|
||||
|
||||
FILES_OVERHEAD = 1
|
||||
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
||||
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
|
||||
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
|
||||
|
||||
|
||||
def random_string(length):
|
||||
letters = string.ascii_letters
|
||||
return ''.join(random.choice(letters) for i in range(length))
|
||||
|
||||
|
||||
def generate_values(date_str, count, sign=1):
|
||||
data = [[date_str, sign * (i + 1), random_string(10)] for i in range(count)]
|
||||
data.sort(key=lambda tup: tup[1])
|
||||
return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data])
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml",
|
||||
"configs/config.d/log_conf.xml"], with_hdfs=True)
|
||||
logging.info("Starting cluster...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
fs.mkdirs('/clickhouse')
|
||||
|
||||
logging.info("Created HDFS directory")
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def wait_for_delete_hdfs_objects(cluster, expected, num_tries=30):
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
while num_tries > 0:
|
||||
num_hdfs_objects = len(fs.listdir('/clickhouse'))
|
||||
if num_hdfs_objects == expected:
|
||||
break;
|
||||
num_tries -= 1
|
||||
time.sleep(1)
|
||||
assert(len(fs.listdir('/clickhouse')) == expected)
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def drop_table(cluster):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
print('Number of hdfs objects to delete:', len(hdfs_objects), sep=' ')
|
||||
|
||||
node.query("DROP TABLE IF EXISTS hdfs_test SYNC")
|
||||
|
||||
try:
|
||||
wait_for_delete_hdfs_objects(cluster, 0)
|
||||
finally:
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
if len(hdfs_objects) == 0:
|
||||
return
|
||||
print("Manually removing extra objects to prevent tests cascade failing: ", hdfs_objects)
|
||||
for path in hdfs_objects:
|
||||
fs.delete(path)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("min_rows_for_wide_part,files_per_part", [(0, FILES_OVERHEAD_PER_PART_WIDE), (8192, FILES_OVERHEAD_PER_PART_COMPACT)])
|
||||
def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part):
|
||||
create_table(cluster, "hdfs_test", additional_settings="min_rows_for_wide_part={}".format(min_rows_for_wide_part))
|
||||
|
||||
node = cluster.instances["node"]
|
||||
|
||||
values1 = generate_values('2020-01-03', 4096)
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(values1))
|
||||
assert node.query("SELECT * FROM hdfs_test order by dt, id FORMAT Values") == values1
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
print(hdfs_objects)
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + files_per_part
|
||||
|
||||
values2 = generate_values('2020-01-04', 4096)
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(values2))
|
||||
assert node.query("SELECT * FROM hdfs_test ORDER BY dt, id FORMAT Values") == values1 + "," + values2
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + files_per_part * 2
|
||||
|
||||
assert node.query("SELECT count(*) FROM hdfs_test where id = 1 FORMAT Values") == "(2)"
|
||||
|
||||
|
||||
def test_alter_table_columns(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096, -1)))
|
||||
|
||||
node.query("ALTER TABLE hdfs_test ADD COLUMN col1 UInt64 DEFAULT 1")
|
||||
# To ensure parts have merged
|
||||
node.query("OPTIMIZE TABLE hdfs_test")
|
||||
|
||||
assert node.query("SELECT sum(col1) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
assert node.query("SELECT sum(col1) FROM hdfs_test WHERE id > 0 FORMAT Values") == "(4096)"
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD_PER_COLUMN)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MODIFY COLUMN col1 String", settings={"mutations_sync": 2})
|
||||
|
||||
assert node.query("SELECT distinct(col1) FROM hdfs_test FORMAT Values") == "('1')"
|
||||
# and file with mutation
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD_PER_COLUMN + 1)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DROP COLUMN col1", settings={"mutations_sync": 2})
|
||||
|
||||
# and 2 files with mutations
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + 2)
|
||||
|
||||
|
||||
def test_attach_detach_partition(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
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
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DETACH PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("ALTER TABLE hdfs_test ATTACH PARTITION '2020-01-03'")
|
||||
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
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DROP PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DETACH PARTITION '2020-01-04'")
|
||||
node.query("ALTER TABLE hdfs_test DROP DETACHED PARTITION '2020-01-04'", settings={"allow_drop_detached": 1})
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD
|
||||
|
||||
|
||||
def test_move_partition_to_another_disk(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
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
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-04' TO DISK 'hdd'")
|
||||
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
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-04' TO DISK 'hdfs'")
|
||||
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
|
||||
|
||||
|
||||
def test_table_manipulations(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
|
||||
node.query("RENAME TABLE hdfs_test TO hdfs_renamed")
|
||||
assert node.query("SELECT count(*) FROM hdfs_renamed FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("RENAME TABLE hdfs_renamed TO hdfs_test")
|
||||
assert node.query("CHECK TABLE hdfs_test FORMAT Values") == "(1)"
|
||||
|
||||
node.query("DETACH TABLE hdfs_test")
|
||||
node.query("ATTACH TABLE hdfs_test")
|
||||
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
|
||||
|
||||
node.query("TRUNCATE TABLE hdfs_test")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD
|
||||
|
||||
|
||||
def test_move_replace_partition_to_another_table(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-05', 4096, -1)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-06', 4096, -1)))
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
||||
create_table(cluster, "hdfs_clone")
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-03' TO TABLE hdfs_clone")
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-05' TO TABLE hdfs_clone")
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
assert node.query("SELECT sum(id) FROM hdfs_clone FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_clone FORMAT Values") == "(8192)"
|
||||
|
||||
# Number of objects in HDFS should be unchanged.
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
||||
# Add new partitions to source table, but with different values and replace them from copied table.
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096, -1)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-05', 4096)))
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 6
|
||||
|
||||
node.query("ALTER TABLE hdfs_test REPLACE PARTITION '2020-01-03' FROM hdfs_clone")
|
||||
node.query("ALTER TABLE hdfs_test REPLACE PARTITION '2020-01-05' FROM hdfs_clone")
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
assert node.query("SELECT sum(id) FROM hdfs_clone FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_clone FORMAT Values") == "(8192)"
|
||||
|
||||
# Wait for outdated partitions deletion.
|
||||
print(1)
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4)
|
||||
|
||||
node.query("DROP TABLE hdfs_clone NO DELAY")
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
# Data should remain in hdfs
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
@ -8,6 +8,18 @@
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
</s31>
|
||||
<s31_again>
|
||||
<type>s3</type>
|
||||
<endpoint>http://minio1:9001/root/data/</endpoint>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
</s31_again>
|
||||
<s32>
|
||||
<type>s3</type>
|
||||
<endpoint>http://minio1:9001/root/data2/</endpoint>
|
||||
<access_key_id>minio</access_key_id>
|
||||
<secret_access_key>minio123</secret_access_key>
|
||||
</s32>
|
||||
</disks>
|
||||
<policies>
|
||||
<s3>
|
||||
@ -28,11 +40,31 @@
|
||||
</volumes>
|
||||
<move_factor>0.0</move_factor>
|
||||
</hybrid>
|
||||
<tiered>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>s31</disk>
|
||||
</main>
|
||||
<external>
|
||||
<disk>s32</disk>
|
||||
</external>
|
||||
</volumes>
|
||||
</tiered>
|
||||
<tiered_copy>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>s31</disk>
|
||||
</main>
|
||||
<external>
|
||||
<disk>s31_again</disk>
|
||||
</external>
|
||||
</volumes>
|
||||
</tiered_copy>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
<min_bytes_for_wide_part>1024</min_bytes_for_wide_part>
|
||||
<old_parts_lifetime>1</old_parts_lifetime>
|
||||
<allow_s3_zero_copy_replication>1</allow_s3_zero_copy_replication>
|
||||
</merge_tree>
|
||||
|
@ -1,3 +1,4 @@
|
||||
import datetime
|
||||
import logging
|
||||
import time
|
||||
|
||||
@ -27,10 +28,10 @@ def cluster():
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def get_large_objects_count(cluster, size=100):
|
||||
def get_large_objects_count(cluster, size=100, folder='data'):
|
||||
minio = cluster.minio_client
|
||||
counter = 0
|
||||
for obj in minio.list_objects(cluster.minio_bucket, 'data/'):
|
||||
for obj in minio.list_objects(cluster.minio_bucket, '{}/'.format(folder)):
|
||||
if obj.size >= size:
|
||||
counter = counter + 1
|
||||
return counter
|
||||
@ -38,11 +39,11 @@ def get_large_objects_count(cluster, size=100):
|
||||
|
||||
def wait_for_large_objects_count(cluster, expected, size=100, timeout=30):
|
||||
while timeout > 0:
|
||||
if get_large_objects_count(cluster, size) == expected:
|
||||
if get_large_objects_count(cluster, size=size) == expected:
|
||||
return
|
||||
timeout -= 1
|
||||
time.sleep(1)
|
||||
assert get_large_objects_count(cluster, size) == expected
|
||||
assert get_large_objects_count(cluster, size=size) == expected
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@ -63,7 +64,7 @@ def test_s3_zero_copy_replication(cluster, policy):
|
||||
)
|
||||
|
||||
node1.query("INSERT INTO s3_test VALUES (0,'data'),(1,'data')")
|
||||
time.sleep(1)
|
||||
node2.query("SYSTEM SYNC REPLICA s3_test")
|
||||
assert node1.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data')"
|
||||
assert node2.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data')"
|
||||
|
||||
@ -71,14 +72,15 @@ def test_s3_zero_copy_replication(cluster, policy):
|
||||
assert get_large_objects_count(cluster) == 1
|
||||
|
||||
node2.query("INSERT INTO s3_test VALUES (2,'data'),(3,'data')")
|
||||
time.sleep(1)
|
||||
node1.query("SYSTEM SYNC REPLICA s3_test")
|
||||
|
||||
assert node2.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data'),(2,'data'),(3,'data')"
|
||||
assert node1.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data'),(2,'data'),(3,'data')"
|
||||
|
||||
# Based on version 20.x - two parts
|
||||
wait_for_large_objects_count(cluster, 2)
|
||||
|
||||
node1.query("OPTIMIZE TABLE s3_test")
|
||||
node1.query("OPTIMIZE TABLE s3_test FINAL")
|
||||
|
||||
# Based on version 20.x - after merge, two old parts and one merged
|
||||
wait_for_large_objects_count(cluster, 3)
|
||||
@ -105,8 +107,7 @@ def test_s3_zero_copy_on_hybrid_storage(cluster):
|
||||
)
|
||||
|
||||
node1.query("INSERT INTO hybrid_test VALUES (0,'data'),(1,'data')")
|
||||
|
||||
time.sleep(1)
|
||||
node2.query("SYSTEM SYNC REPLICA hybrid_test")
|
||||
|
||||
assert node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") == "(0,'data'),(1,'data')"
|
||||
assert node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") == "(0,'data'),(1,'data')"
|
||||
@ -120,7 +121,7 @@ def test_s3_zero_copy_on_hybrid_storage(cluster):
|
||||
assert node2.query("SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values") == "('all','default')"
|
||||
|
||||
# Total objects in S3
|
||||
s3_objects = get_large_objects_count(cluster, 0)
|
||||
s3_objects = get_large_objects_count(cluster, size=0)
|
||||
|
||||
node2.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'")
|
||||
|
||||
@ -135,3 +136,115 @@ def test_s3_zero_copy_on_hybrid_storage(cluster):
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS hybrid_test NO DELAY")
|
||||
node2.query("DROP TABLE IF EXISTS hybrid_test NO DELAY")
|
||||
|
||||
|
||||
def insert_data_time(node, table, number_of_mb, time, start=0):
|
||||
values = ','.join(f"({x},{time})" for x in range(start, int((1024 * 1024 * number_of_mb) / 8) + start + 1))
|
||||
node.query(f"INSERT INTO {table} VALUES {values}")
|
||||
|
||||
|
||||
def insert_large_data(node, table):
|
||||
tm = time.mktime((datetime.date.today() - datetime.timedelta(days=7)).timetuple())
|
||||
insert_data_time(node, table, 1, tm, 0)
|
||||
tm = time.mktime((datetime.date.today() - datetime.timedelta(days=3)).timetuple())
|
||||
insert_data_time(node, table, 1, tm, 1024*1024)
|
||||
tm = time.mktime(datetime.date.today().timetuple())
|
||||
insert_data_time(node, table, 10, tm, 1024*1024*2)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
("storage_policy", "large_data", "iterations"),
|
||||
[
|
||||
("tiered", False, 10),
|
||||
("tiered_copy", False, 10),
|
||||
("tiered", True, 3),
|
||||
("tiered_copy", True, 3),
|
||||
]
|
||||
)
|
||||
def test_s3_zero_copy_with_ttl_move(cluster, storage_policy, large_data, iterations):
|
||||
node1 = cluster.instances["node1"]
|
||||
node2 = cluster.instances["node2"]
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY")
|
||||
node2.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY")
|
||||
|
||||
for i in range(iterations):
|
||||
node1.query(
|
||||
"""
|
||||
CREATE TABLE ttl_move_test ON CLUSTER test_cluster (d UInt64, d1 DateTime)
|
||||
ENGINE=ReplicatedMergeTree('/clickhouse/tables/ttl_move_test', '{}')
|
||||
ORDER BY d
|
||||
TTL d1 + INTERVAL 2 DAY TO VOLUME 'external'
|
||||
SETTINGS storage_policy='{}'
|
||||
"""
|
||||
.format('{replica}', storage_policy)
|
||||
)
|
||||
|
||||
if large_data:
|
||||
insert_large_data(node1, 'ttl_move_test')
|
||||
else:
|
||||
node1.query("INSERT INTO ttl_move_test VALUES (10, now() - INTERVAL 3 DAY)")
|
||||
node1.query("INSERT INTO ttl_move_test VALUES (11, now() - INTERVAL 1 DAY)")
|
||||
|
||||
node1.query("OPTIMIZE TABLE ttl_move_test FINAL")
|
||||
node2.query("SYSTEM SYNC REPLICA ttl_move_test")
|
||||
|
||||
if large_data:
|
||||
assert node1.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(1572867)"
|
||||
assert node2.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(1572867)"
|
||||
else:
|
||||
assert node1.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(2)"
|
||||
assert node2.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(2)"
|
||||
assert node1.query("SELECT d FROM ttl_move_test ORDER BY d FORMAT Values") == "(10),(11)"
|
||||
assert node2.query("SELECT d FROM ttl_move_test ORDER BY d FORMAT Values") == "(10),(11)"
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY")
|
||||
node2.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY")
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
("large_data", "iterations"),
|
||||
[
|
||||
(False, 10),
|
||||
(True, 3),
|
||||
]
|
||||
)
|
||||
def test_s3_zero_copy_with_ttl_delete(cluster, large_data, iterations):
|
||||
node1 = cluster.instances["node1"]
|
||||
node2 = cluster.instances["node2"]
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
|
||||
node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
|
||||
|
||||
for i in range(iterations):
|
||||
node1.query(
|
||||
"""
|
||||
CREATE TABLE ttl_delete_test ON CLUSTER test_cluster (d UInt64, d1 DateTime)
|
||||
ENGINE=ReplicatedMergeTree('/clickhouse/tables/ttl_delete_test', '{}')
|
||||
ORDER BY d
|
||||
TTL d1 + INTERVAL 2 DAY
|
||||
SETTINGS storage_policy='tiered'
|
||||
"""
|
||||
.format('{replica}')
|
||||
)
|
||||
|
||||
if large_data:
|
||||
insert_large_data(node1, 'ttl_delete_test')
|
||||
else:
|
||||
node1.query("INSERT INTO ttl_delete_test VALUES (10, now() - INTERVAL 3 DAY)")
|
||||
node1.query("INSERT INTO ttl_delete_test VALUES (11, now() - INTERVAL 1 DAY)")
|
||||
|
||||
node1.query("OPTIMIZE TABLE ttl_delete_test FINAL")
|
||||
node2.query("SYSTEM SYNC REPLICA ttl_delete_test")
|
||||
|
||||
if large_data:
|
||||
assert node1.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1310721)"
|
||||
assert node2.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1310721)"
|
||||
else:
|
||||
assert node1.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1)"
|
||||
assert node2.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1)"
|
||||
assert node1.query("SELECT d FROM ttl_delete_test ORDER BY d FORMAT Values") == "(11)"
|
||||
assert node2.query("SELECT d FROM ttl_delete_test ORDER BY d FORMAT Values") == "(11)"
|
||||
|
||||
node1.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
|
||||
node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY")
|
||||
|
@ -1750,25 +1750,22 @@ def test_kafka_virtual_columns2(kafka_cluster):
|
||||
producer.send(topic='virt2_0', value=json.dumps({'value': 2}), partition=0, key='k2', timestamp_ms=1577836802002,
|
||||
headers=[('empty_value', b''), ('', b'empty name'), ('', b''), ('repetition', b'1'), ('repetition', b'2')])
|
||||
producer.flush()
|
||||
time.sleep(1)
|
||||
|
||||
producer.send(topic='virt2_0', value=json.dumps({'value': 3}), partition=1, key='k3', timestamp_ms=1577836803003,
|
||||
headers=[('b', b'b'), ('a', b'a')])
|
||||
producer.send(topic='virt2_0', value=json.dumps({'value': 4}), partition=1, key='k4', timestamp_ms=1577836804004,
|
||||
headers=[('a', b'a'), ('b', b'b')])
|
||||
producer.flush()
|
||||
time.sleep(1)
|
||||
|
||||
producer.send(topic='virt2_1', value=json.dumps({'value': 5}), partition=0, key='k5', timestamp_ms=1577836805005)
|
||||
producer.send(topic='virt2_1', value=json.dumps({'value': 6}), partition=0, key='k6', timestamp_ms=1577836806006)
|
||||
producer.flush()
|
||||
time.sleep(1)
|
||||
|
||||
producer.send(topic='virt2_1', value=json.dumps({'value': 7}), partition=1, key='k7', timestamp_ms=1577836807007)
|
||||
producer.send(topic='virt2_1', value=json.dumps({'value': 8}), partition=1, key='k8', timestamp_ms=1577836808008)
|
||||
producer.flush()
|
||||
|
||||
time.sleep(10)
|
||||
instance.wait_for_log_line('kafka.*Committed offset 2.*virt2_[01]', repetitions=4, look_behind_lines=6000)
|
||||
|
||||
members = describe_consumer_group(kafka_cluster, 'virt2')
|
||||
# pprint.pprint(members)
|
||||
@ -1834,8 +1831,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster):
|
||||
1577836804))
|
||||
instance.query("INSERT INTO test.kafka_writer VALUES ({},{},'{}',toDateTime({}))".format(5, 5, 'k5', 1577836805))
|
||||
|
||||
while int(instance.query("SELECT count() FROM test.view")) < 5:
|
||||
time.sleep(1)
|
||||
instance.wait_for_log_line("Committed offset 5")
|
||||
|
||||
result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True)
|
||||
|
||||
|
0
tests/integration/test_version_update/__init__.py
Normal file
0
tests/integration/test_version_update/__init__.py
Normal file
28
tests/integration/test_version_update/test.py
Normal file
28
tests/integration/test_version_update/test.py
Normal file
@ -0,0 +1,28 @@
|
||||
import pytest
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.2', with_installed_binary=True, stay_alive=True)
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def test_modulo_partition_key_after_update(start_cluster):
|
||||
node1.query("CREATE TABLE test (id Int64, v UInt64, value String) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/table1', '1', v) PARTITION BY id % 20 ORDER BY (id, v)")
|
||||
node1.query("INSERT INTO test SELECT number, number, toString(number) FROM numbers(10)")
|
||||
expected = node1.query("SELECT number, number, toString(number) FROM numbers(10)")
|
||||
partition_data = node1.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition")
|
||||
assert(expected == node1.query("SELECT * FROM test ORDER BY id"))
|
||||
node1.restart_with_latest_version(signal=9)
|
||||
assert(expected == node1.query("SELECT * FROM test ORDER BY id"))
|
||||
assert(partition_data == node1.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition"))
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.7">
|
||||
<test max_ignored_relative_change="2">
|
||||
<create_query>CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory</create_query>
|
||||
|
||||
<settings>
|
||||
|
File diff suppressed because one or more lines are too long
@ -0,0 +1,2 @@
|
||||
199
|
||||
57
|
@ -0,0 +1,2 @@
|
||||
SELECT moduloLegacy(199, 200);
|
||||
SELECT moduloLegacy(-199, 200);
|
130
tests/queries/0_stateless/01870_modulo_partition_key.reference
Normal file
130
tests/queries/0_stateless/01870_modulo_partition_key.reference
Normal file
@ -0,0 +1,130 @@
|
||||
simple partition key:
|
||||
-61
|
||||
-60
|
||||
-59
|
||||
-58
|
||||
-57
|
||||
-5
|
||||
-4
|
||||
-3
|
||||
-2
|
||||
-1
|
||||
0
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
57
|
||||
58
|
||||
59
|
||||
60
|
||||
where id % 200 = +-2:
|
||||
-202
|
||||
202
|
||||
where id % 200 > 0:
|
||||
195
|
||||
196
|
||||
197
|
||||
198
|
||||
199
|
||||
201
|
||||
202
|
||||
203
|
||||
204
|
||||
where id % 200 < 0:
|
||||
-205
|
||||
-204
|
||||
-203
|
||||
-202
|
||||
-201
|
||||
-199
|
||||
-198
|
||||
-197
|
||||
-196
|
||||
tuple as partition key:
|
||||
(-1,-1)
|
||||
(-1,0)
|
||||
(-2,-2)
|
||||
(-2,-3)
|
||||
(-2,59)
|
||||
(-2,60)
|
||||
(0,-4)
|
||||
(0,-5)
|
||||
(0,-57)
|
||||
(0,-58)
|
||||
(0,4)
|
||||
(0,57)
|
||||
(0,58)
|
||||
(1,-61)
|
||||
(1,0)
|
||||
(1,1)
|
||||
(2,-59)
|
||||
(2,-60)
|
||||
(2,2)
|
||||
(2,3)
|
||||
recursive modulo partition key:
|
||||
(-1,-1,0)
|
||||
(-2,-2,-1)
|
||||
(-3,-3,-2)
|
||||
(-4,-4,-2)
|
||||
(-5,-5,-2)
|
||||
(-57,-7,-28)
|
||||
(-58,-8,-29)
|
||||
(-59,-9,-30)
|
||||
(-60,0,-30)
|
||||
(-61,-1,-30)
|
||||
(0,0,0)
|
||||
(0,0,0)
|
||||
(1,1,0)
|
||||
(2,2,1)
|
||||
(3,3,2)
|
||||
(4,4,2)
|
||||
(57,7,28)
|
||||
(58,8,29)
|
||||
(59,9,30)
|
||||
(60,0,30)
|
||||
After detach:
|
||||
(-1,-1,0)
|
||||
(-2,-2,-1)
|
||||
(-3,-3,-2)
|
||||
(-4,-4,-2)
|
||||
(-5,-5,-2)
|
||||
(-57,-7,-28)
|
||||
(-58,-8,-29)
|
||||
(-59,-9,-30)
|
||||
(-60,0,-30)
|
||||
(-61,-1,-30)
|
||||
(0,0,0)
|
||||
(0,0,0)
|
||||
(1,1,0)
|
||||
(2,2,1)
|
||||
(3,3,2)
|
||||
(4,4,2)
|
||||
(57,7,28)
|
||||
(58,8,29)
|
||||
(59,9,30)
|
||||
(60,0,30)
|
||||
Indexes:
|
||||
100
|
||||
comparison:
|
||||
0 -205 -5 -5
|
||||
1 -204 -4 -4
|
||||
2 -203 -3 -3
|
||||
3 -202 -2 -2
|
||||
4 -201 -1 -1
|
||||
5 -200 0 0
|
||||
6 -199 -199 57
|
||||
7 -198 -198 58
|
||||
8 -197 -197 59
|
||||
9 -196 -196 60
|
||||
400 195 195 -61
|
||||
401 196 196 -60
|
||||
402 197 197 -59
|
||||
403 198 198 -58
|
||||
404 199 199 -57
|
||||
405 200 0 0
|
||||
406 201 1 1
|
||||
407 202 2 2
|
||||
408 203 3 3
|
||||
409 204 4 4
|
50
tests/queries/0_stateless/01870_modulo_partition_key.sql
Normal file
50
tests/queries/0_stateless/01870_modulo_partition_key.sql
Normal file
@ -0,0 +1,50 @@
|
||||
SELECT 'simple partition key:';
|
||||
DROP TABLE IF EXISTS table1 SYNC;
|
||||
CREATE TABLE table1 (id Int64, v UInt64)
|
||||
ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/table12', '1', v)
|
||||
PARTITION BY id % 200 ORDER BY id;
|
||||
INSERT INTO table1 SELECT number-205, number FROM numbers(10);
|
||||
INSERT INTO table1 SELECT number-205, number FROM numbers(400, 10);
|
||||
SELECT toInt64(partition) as p FROM system.parts WHERE table='table1' and database=currentDatabase() ORDER BY p;
|
||||
|
||||
select 'where id % 200 = +-2:';
|
||||
select id from table1 where id % 200 = 2 OR id % 200 = -2 order by id;
|
||||
select 'where id % 200 > 0:';
|
||||
select id from table1 where id % 200 > 0 order by id;
|
||||
select 'where id % 200 < 0:';
|
||||
select id from table1 where id % 200 < 0 order by id;
|
||||
|
||||
SELECT 'tuple as partition key:';
|
||||
DROP TABLE IF EXISTS table2;
|
||||
CREATE TABLE table2 (id Int64, v UInt64)
|
||||
ENGINE = MergeTree()
|
||||
PARTITION BY (toInt32(id / 2) % 3, id % 200) ORDER BY id;
|
||||
INSERT INTO table2 SELECT number-205, number FROM numbers(10);
|
||||
INSERT INTO table2 SELECT number-205, number FROM numbers(400, 10);
|
||||
SELECT partition as p FROM system.parts WHERE table='table2' and database=currentDatabase() ORDER BY p;
|
||||
|
||||
SELECT 'recursive modulo partition key:';
|
||||
DROP TABLE IF EXISTS table3;
|
||||
CREATE TABLE table3 (id Int64, v UInt64)
|
||||
ENGINE = MergeTree()
|
||||
PARTITION BY (id % 200, (id % 200) % 10, toInt32(round((id % 200) / 2, 0))) ORDER BY id;
|
||||
INSERT INTO table3 SELECT number-205, number FROM numbers(10);
|
||||
INSERT INTO table3 SELECT number-205, number FROM numbers(400, 10);
|
||||
SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p;
|
||||
|
||||
DETACH TABLE table3;
|
||||
ATTACH TABLE table3;
|
||||
SELECT 'After detach:';
|
||||
SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p;
|
||||
|
||||
SELECT 'Indexes:';
|
||||
DROP TABLE IF EXISTS table4;
|
||||
CREATE TABLE table4 (id Int64, v UInt64, s String,
|
||||
INDEX a (id * 2, s) TYPE minmax GRANULARITY 3
|
||||
) ENGINE = MergeTree() PARTITION BY id % 10 ORDER BY v;
|
||||
INSERT INTO table4 SELECT number, number, toString(number) FROM numbers(1000);
|
||||
SELECT count() FROM table4 WHERE id % 10 = 7;
|
||||
|
||||
SELECT 'comparison:';
|
||||
SELECT v, v-205 as vv, modulo(vv, 200), moduloLegacy(vv, 200) FROM table1 ORDER BY v;
|
||||
|
@ -0,0 +1,4 @@
|
||||
1 2020-11-11
|
||||
1 2021-01-01
|
||||
1 2020-11-11
|
||||
1 2021-01-01
|
11
tests/queries/0_stateless/01881_to_week_monotonic_fix.sql
Normal file
11
tests/queries/0_stateless/01881_to_week_monotonic_fix.sql
Normal file
@ -0,0 +1,11 @@
|
||||
drop table if exists test_tbl;
|
||||
|
||||
create table test_tbl (vend_nm String, ship_dt Date) engine MergeTree partition by toWeek(ship_dt) order by vend_nm;
|
||||
|
||||
insert into test_tbl values('1', '2020-11-11'), ('1', '2021-01-01');
|
||||
|
||||
select * From test_tbl where ship_dt >= toDate('2020-11-01') and ship_dt <= toDate('2021-05-05') order by ship_dt;
|
||||
|
||||
select * From test_tbl where ship_dt >= toDate('2020-01-01') and ship_dt <= toDate('2021-05-05') order by ship_dt;
|
||||
|
||||
drop table test_tbl;
|
@ -236,4 +236,5 @@
|
||||
01801_s3_distributed
|
||||
01833_test_collation_alvarotuso
|
||||
01850_dist_INSERT_preserve_error
|
||||
01870_modulo_partition_key
|
||||
01880_remote_ipv6
|
||||
|
@ -272,6 +272,7 @@
|
||||
"01145_with_fill_const",
|
||||
"01149_zookeeper_mutation_stuck_after_replace_partition",
|
||||
"01150_ddl_guard_rwr",
|
||||
"01155_old_mutation_parts_to_do",
|
||||
"01185_create_or_replace_table",
|
||||
"01187_set_profile_as_setting",
|
||||
"01188_attach_table_from_path",
|
||||
@ -723,6 +724,7 @@
|
||||
"01850_dist_INSERT_preserve_error", // uses cluster with different static databases shard_0/shard_1
|
||||
"01821_table_comment",
|
||||
"01710_projection_fetch",
|
||||
"01870_modulo_partition_key",
|
||||
"01870_buffer_flush" // creates database
|
||||
]
|
||||
}
|
||||
|
118
tests/testflows/extended_precision_data_types/common.py
Normal file
118
tests/testflows/extended_precision_data_types/common.py
Normal file
@ -0,0 +1,118 @@
|
||||
import uuid
|
||||
|
||||
from contextlib import contextmanager
|
||||
|
||||
from testflows.core.name import basename, parentname
|
||||
from testflows._core.testtype import TestSubType
|
||||
from testflows.asserts import values, error, snapshot
|
||||
|
||||
from helpers.common import *
|
||||
|
||||
@contextmanager
|
||||
def allow_experimental_bigint(node):
|
||||
"""Enable experimental big int setting in Clickhouse.
|
||||
"""
|
||||
setting = ("allow_experimental_bigint_types", 1)
|
||||
default_query_settings = None
|
||||
|
||||
try:
|
||||
with Given("I add allow_experimental_bigint to the default query settings"):
|
||||
default_query_settings = getsattr(current().context, "default_query_settings", [])
|
||||
default_query_settings.append(setting)
|
||||
yield
|
||||
finally:
|
||||
with Finally("I remove allow_experimental_bigint from the default query settings"):
|
||||
if default_query_settings:
|
||||
try:
|
||||
default_query_settings.pop(default_query_settings.index(setting))
|
||||
except ValueError:
|
||||
pass
|
||||
|
||||
@TestStep(Given)
|
||||
def allow_experimental_map_type(self):
|
||||
"""Set allow_experimental_map_type = 1
|
||||
"""
|
||||
setting = ("allow_experimental_map_type", 1)
|
||||
default_query_settings = None
|
||||
|
||||
try:
|
||||
with By("adding allow_experimental_map_type to the default query settings"):
|
||||
default_query_settings = getsattr(current().context, "default_query_settings", [])
|
||||
default_query_settings.append(setting)
|
||||
yield
|
||||
finally:
|
||||
with Finally("I remove allow_experimental_map_type from the default query settings"):
|
||||
if default_query_settings:
|
||||
try:
|
||||
default_query_settings.pop(default_query_settings.index(setting))
|
||||
except ValueError:
|
||||
pass
|
||||
|
||||
def execute_query(sql, expected=None, format="TabSeparatedWithNames", compare_func=None):
|
||||
"""Execute SQL query and compare the output to the snapshot.
|
||||
"""
|
||||
name = basename(current().name)
|
||||
|
||||
with When("I execute query", description=sql):
|
||||
r = current().context.node.query(sql + " FORMAT " + format)
|
||||
|
||||
if expected is not None:
|
||||
with Then("I check output against expected"):
|
||||
|
||||
if compare_func is None:
|
||||
assert r.output.strip() == expected, error()
|
||||
|
||||
else:
|
||||
assert compare_func(r.output.strip(), expected), error()
|
||||
|
||||
else:
|
||||
with Then("I check output against snapshot"):
|
||||
with values() as that:
|
||||
assert that(snapshot("\n" + r.output.strip() + "\n", "tests", name=name, encoder=str)), error()
|
||||
|
||||
@TestStep(Given)
|
||||
def table(self, data_type, name="table0"):
|
||||
"""Create a table.
|
||||
"""
|
||||
node = current().context.node
|
||||
|
||||
try:
|
||||
with By("creating table"):
|
||||
node.query(f"CREATE TABLE {name}(a {data_type}) ENGINE = Memory")
|
||||
yield
|
||||
|
||||
finally:
|
||||
with Finally("drop the table"):
|
||||
node.query(f"DROP TABLE IF EXISTS {name}")
|
||||
|
||||
def getuid():
|
||||
"""Create a unique variable name based on the test it is called from.
|
||||
"""
|
||||
if current().subtype == TestSubType.Example:
|
||||
testname = f"{basename(parentname(current().name)).replace(' ', '_').replace(',','')}"
|
||||
else:
|
||||
testname = f"{basename(current().name).replace(' ', '_').replace(',','')}"
|
||||
|
||||
for char in ['(', ')', '[', ']','\'']:
|
||||
testname = testname.replace(f'{char}', '')
|
||||
|
||||
return testname + "_" + str(uuid.uuid1()).replace('-', '_')
|
||||
|
||||
def to_data_type(data_type, value):
|
||||
"""Return a conversion statement based on the data type provided
|
||||
"""
|
||||
if data_type in ['Decimal256(0)']:
|
||||
return f'toDecimal256(\'{value}\',0)'
|
||||
|
||||
else:
|
||||
return f'to{data_type}(\'{value}\')'
|
||||
|
||||
|
||||
data_types = [
|
||||
('Int128', '-170141183460469231731687303715884105728', '170141183460469231731687303715884105727'),
|
||||
('Int256', '-57896044618658097711785492504343953926634992332820282019728792003956564819968', '57896044618658097711785492504343953926634992332820282019728792003956564819967'),
|
||||
('UInt128','0','340282366920938463463374607431768211455'),
|
||||
('UInt256', '0', '115792089237316195423570985008687907853269984665640564039457584007913129639935'),
|
||||
]
|
||||
|
||||
Decimal256_min_max = -1000000000000000000000000000000000000000000000000000000000000000000000000000,1000000000000000000000000000000000000000000000000000000000000000000000000000
|
@ -0,0 +1,6 @@
|
||||
<yandex>
|
||||
<timezone>Europe/Moscow</timezone>
|
||||
<listen_host replace="replace">::</listen_host>
|
||||
<path>/var/lib/clickhouse/</path>
|
||||
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
|
||||
</yandex>
|
@ -0,0 +1,17 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
<part_log>
|
||||
<database>system</database>
|
||||
<table>part_log</table>
|
||||
<flush_interval_milliseconds>500</flush_interval_milliseconds>
|
||||
</part_log>
|
||||
</yandex>
|
@ -0,0 +1,20 @@
|
||||
<yandex>
|
||||
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<default>
|
||||
<keep_free_space_bytes>1024</keep_free_space_bytes>
|
||||
</default>
|
||||
</disks>
|
||||
<policies>
|
||||
<default>
|
||||
<volumes>
|
||||
<default>
|
||||
<disk>default</disk>
|
||||
</default>
|
||||
</volumes>
|
||||
</default>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
</yandex>
|
@ -0,0 +1,448 @@
|
||||
<?xml version="1.0"?>
|
||||
<!--
|
||||
NOTE: User and query level settings are set up in "users.xml" file.
|
||||
-->
|
||||
<yandex>
|
||||
<logger>
|
||||
<!-- Possible levels: https://github.com/pocoproject/poco/blob/develop/Foundation/include/Poco/Logger.h#L105 -->
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<!-- <console>1</console> --> <!-- Default behavior is autodetection (log to console if not daemon mode and is tty) -->
|
||||
</logger>
|
||||
<!--display_name>production</display_name--> <!-- It is the name that will be shown in the client -->
|
||||
<http_port>8123</http_port>
|
||||
<tcp_port>9000</tcp_port>
|
||||
|
||||
<!-- For HTTPS and SSL over native protocol. -->
|
||||
<!--
|
||||
<https_port>8443</https_port>
|
||||
<tcp_port_secure>9440</tcp_port_secure>
|
||||
-->
|
||||
|
||||
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
|
||||
<openSSL>
|
||||
<server> <!-- Used for https server AND secure tcp port -->
|
||||
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
|
||||
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
|
||||
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
|
||||
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
</server>
|
||||
|
||||
<client> <!-- Used for connecting to https dictionary source -->
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
<!-- Use for self-signed: <verificationMode>none</verificationMode> -->
|
||||
<invalidCertificateHandler>
|
||||
<!-- Use for self-signed: <name>AcceptCertificateHandler</name> -->
|
||||
<name>RejectCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</client>
|
||||
</openSSL>
|
||||
|
||||
<!-- Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 -->
|
||||
<!--
|
||||
<http_server_default_response><![CDATA[<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>]]></http_server_default_response>
|
||||
-->
|
||||
|
||||
<!-- Port for communication between replicas. Used for data exchange. -->
|
||||
<interserver_http_port>9009</interserver_http_port>
|
||||
|
||||
<!-- Hostname that is used by other replicas to request this server.
|
||||
If not specified, than it is determined analoguous to 'hostname -f' command.
|
||||
This setting could be used to switch replication to another network interface.
|
||||
-->
|
||||
<!--
|
||||
<interserver_http_host>example.yandex.ru</interserver_http_host>
|
||||
-->
|
||||
|
||||
<!-- Listen specified host. use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. -->
|
||||
<!-- <listen_host>::</listen_host> -->
|
||||
<!-- Same for hosts with disabled ipv6: -->
|
||||
<listen_host>0.0.0.0</listen_host>
|
||||
|
||||
<!-- Default values - try listen localhost on ipv4 and ipv6: -->
|
||||
<!--
|
||||
<listen_host>::1</listen_host>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
-->
|
||||
<!-- Don't exit if ipv6 or ipv4 unavailable, but listen_host with this protocol specified -->
|
||||
<!-- <listen_try>0</listen_try> -->
|
||||
|
||||
<!-- Allow listen on same address:port -->
|
||||
<!-- <listen_reuse_port>0</listen_reuse_port> -->
|
||||
|
||||
<!-- <listen_backlog>64</listen_backlog> -->
|
||||
|
||||
<max_connections>4096</max_connections>
|
||||
<keep_alive_timeout>3</keep_alive_timeout>
|
||||
|
||||
<!-- Maximum number of concurrent queries. -->
|
||||
<max_concurrent_queries>100</max_concurrent_queries>
|
||||
|
||||
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
|
||||
correct maximum value. -->
|
||||
<!-- <max_open_files>262144</max_open_files> -->
|
||||
|
||||
<!-- Size of cache of uncompressed blocks of data, used in tables of MergeTree family.
|
||||
In bytes. Cache is single for server. Memory is allocated only on demand.
|
||||
Cache is used when 'use_uncompressed_cache' user setting turned on (off by default).
|
||||
Uncompressed cache is advantageous only for very short queries and in rare cases.
|
||||
-->
|
||||
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
|
||||
|
||||
<!-- Approximate size of mark cache, used in tables of MergeTree family.
|
||||
In bytes. Cache is single for server. Memory is allocated only on demand.
|
||||
You should not lower this value.
|
||||
-->
|
||||
<mark_cache_size>5368709120</mark_cache_size>
|
||||
|
||||
|
||||
<!-- Path to data directory, with trailing slash. -->
|
||||
<path>/var/lib/clickhouse/</path>
|
||||
|
||||
<!-- Path to temporary data for processing hard queries. -->
|
||||
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
|
||||
|
||||
<!-- Directory with user provided files that are accessible by 'file' table function. -->
|
||||
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
|
||||
|
||||
<!-- Path to folder where users and roles created by SQL commands are stored. -->
|
||||
<access_control_path>/var/lib/clickhouse/access/</access_control_path>
|
||||
|
||||
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
|
||||
<user_directories>
|
||||
<users_xml>
|
||||
<!-- Path to configuration file with predefined users. -->
|
||||
<path>users.xml</path>
|
||||
</users_xml>
|
||||
<local_directory>
|
||||
<!-- Path to folder where users created by SQL commands are stored. -->
|
||||
<path>/var/lib/clickhouse/access/</path>
|
||||
</local_directory>
|
||||
</user_directories>
|
||||
|
||||
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
|
||||
<users_config>users.xml</users_config>
|
||||
|
||||
<!-- Default profile of settings. -->
|
||||
<default_profile>default</default_profile>
|
||||
|
||||
<!-- System profile of settings. This settings are used by internal processes (Buffer storage, Distibuted DDL worker and so on). -->
|
||||
<!-- <system_profile>default</system_profile> -->
|
||||
|
||||
<!-- Default database. -->
|
||||
<default_database>default</default_database>
|
||||
|
||||
<!-- Server time zone could be set here.
|
||||
|
||||
Time zone is used when converting between String and DateTime types,
|
||||
when printing DateTime in text formats and parsing DateTime from text,
|
||||
it is used in date and time related functions, if specific time zone was not passed as an argument.
|
||||
|
||||
Time zone is specified as identifier from IANA time zone database, like UTC or Africa/Abidjan.
|
||||
If not specified, system time zone at server startup is used.
|
||||
|
||||
Please note, that server could display time zone alias instead of specified name.
|
||||
Example: W-SU is an alias for Europe/Moscow and Zulu is an alias for UTC.
|
||||
-->
|
||||
<!-- <timezone>Europe/Moscow</timezone> -->
|
||||
|
||||
<!-- You can specify umask here (see "man umask"). Server will apply it on startup.
|
||||
Number is always parsed as octal. Default umask is 027 (other users cannot read logs, data files, etc; group can only read).
|
||||
-->
|
||||
<!-- <umask>022</umask> -->
|
||||
|
||||
<!-- Perform mlockall after startup to lower first queries latency
|
||||
and to prevent clickhouse executable from being paged out under high IO load.
|
||||
Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
|
||||
-->
|
||||
<mlock_executable>false</mlock_executable>
|
||||
|
||||
<!-- Configuration of clusters that could be used in Distributed tables.
|
||||
https://clickhouse.yandex/docs/en/table_engines/distributed/
|
||||
-->
|
||||
<remote_servers incl="clickhouse_remote_servers" >
|
||||
<!-- Test only shard config for testing distributed storage -->
|
||||
<test_shard_localhost>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost>
|
||||
<test_cluster_two_shards_localhost>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_shards_localhost>
|
||||
<test_shard_localhost_secure>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9440</port>
|
||||
<secure>1</secure>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost_secure>
|
||||
<test_unavailable_shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>1</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_unavailable_shard>
|
||||
</remote_servers>
|
||||
|
||||
|
||||
<!-- If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file.
|
||||
By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element.
|
||||
Values for substitutions are specified in /yandex/name_of_substitution elements in that file.
|
||||
-->
|
||||
|
||||
<!-- ZooKeeper is used to store metadata about replicas, when using Replicated tables.
|
||||
Optional. If you don't use replicated tables, you could omit that.
|
||||
|
||||
See https://clickhouse.yandex/docs/en/table_engines/replication/
|
||||
-->
|
||||
<zookeeper incl="zookeeper-servers" optional="true" />
|
||||
|
||||
<!-- Substitutions for parameters of replicated tables.
|
||||
Optional. If you don't use replicated tables, you could omit that.
|
||||
|
||||
See https://clickhouse.yandex/docs/en/table_engines/replication/#creating-replicated-tables
|
||||
-->
|
||||
<macros incl="macros" optional="true" />
|
||||
|
||||
|
||||
<!-- Reloading interval for embedded dictionaries, in seconds. Default: 3600. -->
|
||||
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
|
||||
|
||||
|
||||
<!-- Maximum session timeout, in seconds. Default: 3600. -->
|
||||
<max_session_timeout>3600</max_session_timeout>
|
||||
|
||||
<!-- Default session timeout, in seconds. Default: 60. -->
|
||||
<default_session_timeout>60</default_session_timeout>
|
||||
|
||||
<!-- Sending data to Graphite for monitoring. Several sections can be defined. -->
|
||||
<!--
|
||||
interval - send every X second
|
||||
root_path - prefix for keys
|
||||
hostname_in_path - append hostname to root_path (default = true)
|
||||
metrics - send data from table system.metrics
|
||||
events - send data from table system.events
|
||||
asynchronous_metrics - send data from table system.asynchronous_metrics
|
||||
-->
|
||||
<!--
|
||||
<graphite>
|
||||
<host>localhost</host>
|
||||
<port>42000</port>
|
||||
<timeout>0.1</timeout>
|
||||
<interval>60</interval>
|
||||
<root_path>one_min</root_path>
|
||||
<hostname_in_path>true</hostname_in_path>
|
||||
|
||||
<metrics>true</metrics>
|
||||
<events>true</events>
|
||||
<asynchronous_metrics>true</asynchronous_metrics>
|
||||
</graphite>
|
||||
<graphite>
|
||||
<host>localhost</host>
|
||||
<port>42000</port>
|
||||
<timeout>0.1</timeout>
|
||||
<interval>1</interval>
|
||||
<root_path>one_sec</root_path>
|
||||
|
||||
<metrics>true</metrics>
|
||||
<events>true</events>
|
||||
<asynchronous_metrics>false</asynchronous_metrics>
|
||||
</graphite>
|
||||
-->
|
||||
|
||||
|
||||
<!-- Query log. Used only for queries with setting log_queries = 1. -->
|
||||
<query_log>
|
||||
<!-- What table to insert data. If table is not exist, it will be created.
|
||||
When query log structure is changed after system update,
|
||||
then old table will be renamed and new table will be created automatically.
|
||||
-->
|
||||
<database>system</database>
|
||||
<table>query_log</table>
|
||||
<!--
|
||||
PARTITION BY expr https://clickhouse.yandex/docs/en/table_engines/custom_partitioning_key/
|
||||
Example:
|
||||
event_date
|
||||
toMonday(event_date)
|
||||
toYYYYMM(event_date)
|
||||
toStartOfHour(event_time)
|
||||
-->
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<!-- Interval of flushing data. -->
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</query_log>
|
||||
|
||||
<!-- Trace log. Stores stack traces collected by query profilers.
|
||||
See query_profiler_real_time_period_ns and query_profiler_cpu_time_period_ns settings. -->
|
||||
<trace_log>
|
||||
<database>system</database>
|
||||
<table>trace_log</table>
|
||||
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</trace_log>
|
||||
|
||||
<!-- Query thread log. Has information about all threads participated in query execution.
|
||||
Used only for queries with setting log_query_threads = 1. -->
|
||||
<query_thread_log>
|
||||
<database>system</database>
|
||||
<table>query_thread_log</table>
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</query_thread_log>
|
||||
|
||||
<!-- Uncomment if use part log.
|
||||
Part log contains information about all actions with parts in MergeTree tables (creation, deletion, merges, downloads).
|
||||
<part_log>
|
||||
<database>system</database>
|
||||
<table>part_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</part_log>
|
||||
-->
|
||||
|
||||
<!-- Uncomment to write text log into table.
|
||||
Text log contains all information from usual server log but stores it in structured and efficient way.
|
||||
<text_log>
|
||||
<database>system</database>
|
||||
<table>text_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</text_log>
|
||||
-->
|
||||
|
||||
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
|
||||
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/
|
||||
-->
|
||||
|
||||
<!-- Path to file with region hierarchy. -->
|
||||
<!-- <path_to_regions_hierarchy_file>/opt/geo/regions_hierarchy.txt</path_to_regions_hierarchy_file> -->
|
||||
|
||||
<!-- Path to directory with files containing names of regions -->
|
||||
<!-- <path_to_regions_names_files>/opt/geo/</path_to_regions_names_files> -->
|
||||
|
||||
|
||||
<!-- Configuration of external dictionaries. See:
|
||||
https://clickhouse.yandex/docs/en/dicts/external_dicts/
|
||||
-->
|
||||
<dictionaries_config>*_dictionary.xml</dictionaries_config>
|
||||
|
||||
<!-- Uncomment if you want data to be compressed 30-100% better.
|
||||
Don't do that if you just started using ClickHouse.
|
||||
-->
|
||||
<compression incl="clickhouse_compression">
|
||||
<!--
|
||||
<!- - Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - ->
|
||||
<case>
|
||||
|
||||
<!- - Conditions. All must be satisfied. Some conditions may be omitted. - ->
|
||||
<min_part_size>10000000000</min_part_size> <!- - Min part size in bytes. - ->
|
||||
<min_part_size_ratio>0.01</min_part_size_ratio> <!- - Min size of part relative to whole table size. - ->
|
||||
|
||||
<!- - What compression method to use. - ->
|
||||
<method>zstd</method>
|
||||
</case>
|
||||
-->
|
||||
</compression>
|
||||
|
||||
<!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster.
|
||||
Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. -->
|
||||
<distributed_ddl>
|
||||
<!-- Path in ZooKeeper to queue with DDL queries -->
|
||||
<path>/clickhouse/task_queue/ddl</path>
|
||||
|
||||
<!-- Settings from this profile will be used to execute DDL queries -->
|
||||
<!-- <profile>default</profile> -->
|
||||
</distributed_ddl>
|
||||
|
||||
<!-- Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
|
||||
<!--
|
||||
<merge_tree>
|
||||
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
|
||||
</merge_tree>
|
||||
-->
|
||||
|
||||
<!-- Protection from accidental DROP.
|
||||
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
|
||||
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
|
||||
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
|
||||
The same for max_partition_size_to_drop.
|
||||
Uncomment to disable protection.
|
||||
-->
|
||||
<!-- <max_table_size_to_drop>0</max_table_size_to_drop> -->
|
||||
<!-- <max_partition_size_to_drop>0</max_partition_size_to_drop> -->
|
||||
|
||||
<!-- Example of parameters for GraphiteMergeTree table engine -->
|
||||
<graphite_rollup_example>
|
||||
<pattern>
|
||||
<regexp>click_cost</regexp>
|
||||
<function>any</function>
|
||||
<retention>
|
||||
<age>0</age>
|
||||
<precision>3600</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>86400</age>
|
||||
<precision>60</precision>
|
||||
</retention>
|
||||
</pattern>
|
||||
<default>
|
||||
<function>max</function>
|
||||
<retention>
|
||||
<age>0</age>
|
||||
<precision>60</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>3600</age>
|
||||
<precision>300</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>86400</age>
|
||||
<precision>3600</precision>
|
||||
</retention>
|
||||
</default>
|
||||
</graphite_rollup_example>
|
||||
|
||||
<!-- Directory in <clickhouse-path> containing schema files for various input formats.
|
||||
The directory will be created if it doesn't exist.
|
||||
-->
|
||||
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
|
||||
|
||||
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
|
||||
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
|
||||
</yandex>
|
@ -0,0 +1,133 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<!-- Profiles of settings. -->
|
||||
<profiles>
|
||||
<!-- Default settings. -->
|
||||
<default>
|
||||
<!-- Maximum memory usage for processing single query, in bytes. -->
|
||||
<max_memory_usage>10000000000</max_memory_usage>
|
||||
|
||||
<!-- Use cache of uncompressed blocks of data. Meaningful only for processing many of very short queries. -->
|
||||
<use_uncompressed_cache>0</use_uncompressed_cache>
|
||||
|
||||
<!-- How to choose between replicas during distributed query processing.
|
||||
random - choose random replica from set of replicas with minimum number of errors
|
||||
nearest_hostname - from set of replicas with minimum number of errors, choose replica
|
||||
with minimum number of different symbols between replica's hostname and local hostname
|
||||
(Hamming distance).
|
||||
in_order - first live replica is chosen in specified order.
|
||||
first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors.
|
||||
-->
|
||||
<load_balancing>random</load_balancing>
|
||||
</default>
|
||||
|
||||
<!-- Profile that allows only read queries. -->
|
||||
<readonly>
|
||||
<readonly>1</readonly>
|
||||
</readonly>
|
||||
</profiles>
|
||||
|
||||
<!-- Users and ACL. -->
|
||||
<users>
|
||||
<!-- If user name was not specified, 'default' user is used. -->
|
||||
<default>
|
||||
<!-- Password could be specified in plaintext or in SHA256 (in hex format).
|
||||
|
||||
If you want to specify password in plaintext (not recommended), place it in 'password' element.
|
||||
Example: <password>qwerty</password>.
|
||||
Password could be empty.
|
||||
|
||||
If you want to specify SHA256, place it in 'password_sha256_hex' element.
|
||||
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
|
||||
Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019).
|
||||
|
||||
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
|
||||
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
|
||||
|
||||
How to generate decent password:
|
||||
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
|
||||
In first line will be password and in second - corresponding SHA256.
|
||||
|
||||
How to generate double SHA1:
|
||||
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1
|
||||
In first line will be password and in second - corresponding double SHA1.
|
||||
-->
|
||||
<password></password>
|
||||
|
||||
<!-- List of networks with open access.
|
||||
|
||||
To open access from everywhere, specify:
|
||||
<ip>::/0</ip>
|
||||
|
||||
To open access only from localhost, specify:
|
||||
<ip>::1</ip>
|
||||
<ip>127.0.0.1</ip>
|
||||
|
||||
Each element of list has one of the following forms:
|
||||
<ip> IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0
|
||||
2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::.
|
||||
<host> Hostname. Example: server01.yandex.ru.
|
||||
To check access, DNS query is performed, and all received addresses compared to peer address.
|
||||
<host_regexp> Regular expression for host names. Example, ^server\d\d-\d\d-\d\.yandex\.ru$
|
||||
To check access, DNS PTR query is performed for peer address and then regexp is applied.
|
||||
Then, for result of PTR query, another DNS query is performed and all received addresses compared to peer address.
|
||||
Strongly recommended that regexp is ends with $
|
||||
All results of DNS requests are cached till server restart.
|
||||
-->
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
|
||||
<!-- Settings profile for user. -->
|
||||
<profile>default</profile>
|
||||
|
||||
<!-- Quota for user. -->
|
||||
<quota>default</quota>
|
||||
|
||||
<!-- Allow access management -->
|
||||
<access_management>1</access_management>
|
||||
|
||||
<!-- Example of row level security policy. -->
|
||||
<!-- <databases>
|
||||
<test>
|
||||
<filtered_table1>
|
||||
<filter>a = 1</filter>
|
||||
</filtered_table1>
|
||||
<filtered_table2>
|
||||
<filter>a + b < 1 or c - d > 5</filter>
|
||||
</filtered_table2>
|
||||
</test>
|
||||
</databases> -->
|
||||
</default>
|
||||
|
||||
<!-- Example of user with readonly access. -->
|
||||
<!-- <readonly>
|
||||
<password></password>
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::1</ip>
|
||||
<ip>127.0.0.1</ip>
|
||||
</networks>
|
||||
<profile>readonly</profile>
|
||||
<quota>default</quota>
|
||||
</readonly> -->
|
||||
</users>
|
||||
|
||||
<!-- Quotas. -->
|
||||
<quotas>
|
||||
<!-- Name of quota. -->
|
||||
<default>
|
||||
<!-- Limits for time interval. You could specify many intervals with different limits. -->
|
||||
<interval>
|
||||
<!-- Length of interval. -->
|
||||
<duration>3600</duration>
|
||||
|
||||
<!-- No limits. Just calculate resource usage for time interval. -->
|
||||
<queries>0</queries>
|
||||
<errors>0</errors>
|
||||
<result_rows>0</result_rows>
|
||||
<read_rows>0</read_rows>
|
||||
<execution_time>0</execution_time>
|
||||
</interval>
|
||||
</default>
|
||||
</quotas>
|
||||
</yandex>
|
11
tests/testflows/extended_precision_data_types/errors.py
Normal file
11
tests/testflows/extended_precision_data_types/errors.py
Normal file
@ -0,0 +1,11 @@
|
||||
def not_implemented_bigints(name):
|
||||
return(48, f"Exception: {name} is not implemented for big integers")
|
||||
|
||||
def bigints_not_implements(name):
|
||||
return(48, f'Exception: {name} for big integers is not implemented')
|
||||
|
||||
def illegal_type():
|
||||
return(43, 'Exception: Illegal type')
|
||||
|
||||
def illegal_column():
|
||||
return(44, 'Exception: Illegal column')
|
@ -0,0 +1,27 @@
|
||||
version: '2.3'
|
||||
|
||||
services:
|
||||
clickhouse:
|
||||
image: yandex/clickhouse-integration-test
|
||||
expose:
|
||||
- "9000"
|
||||
- "9009"
|
||||
- "8123"
|
||||
volumes:
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.d:/etc/clickhouse-server/config.d"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.d:/etc/clickhouse-server/users.d"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.xml:/etc/clickhouse-server/config.xml"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.xml:/etc/clickhouse-server/users.xml"
|
||||
- "${CLICKHOUSE_TESTS_SERVER_BIN_PATH:-/usr/bin/clickhouse}:/usr/bin/clickhouse"
|
||||
- "${CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH:-/usr/bin/clickhouse-odbc-bridge}:/usr/bin/clickhouse-odbc-bridge"
|
||||
entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log"
|
||||
healthcheck:
|
||||
test: clickhouse client --query='select 1'
|
||||
interval: 10s
|
||||
timeout: 10s
|
||||
retries: 3
|
||||
start_period: 300s
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
security_opt:
|
||||
- label:disable
|
@ -0,0 +1,30 @@
|
||||
version: '2.3'
|
||||
|
||||
services:
|
||||
|
||||
mysql1:
|
||||
extends:
|
||||
file: mysql-service.yml
|
||||
service: mysql
|
||||
hostname: mysql1
|
||||
volumes:
|
||||
- "${CLICKHOUSE_TESTS_DIR}/_instances/mysql1/database:/var/lib/mysql"
|
||||
|
||||
clickhouse1:
|
||||
extends:
|
||||
file: clickhouse-service.yml
|
||||
service: clickhouse
|
||||
hostname: clickhouse1
|
||||
volumes:
|
||||
- "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/database/:/var/lib/clickhouse/"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/logs/:/var/log/clickhouse-server/"
|
||||
|
||||
# dummy service which does nothing, but allows to postpone
|
||||
# 'docker-compose up -d' till all dependecies will go healthy
|
||||
all_services_ready:
|
||||
image: hello-world
|
||||
depends_on:
|
||||
clickhouse1:
|
||||
condition: service_healthy
|
||||
mysql1:
|
||||
condition: service_healthy
|
@ -0,0 +1,19 @@
|
||||
version: '2.3'
|
||||
|
||||
services:
|
||||
mysql:
|
||||
image: mysql:5.7.30
|
||||
restart: always
|
||||
environment:
|
||||
MYSQL_DATABASE: 'db'
|
||||
MYSQL_USER: 'user'
|
||||
MYSQL_PASSWORD: 'password'
|
||||
MYSQL_ROOT_PASSWORD: 'password'
|
||||
expose:
|
||||
- '3306'
|
||||
healthcheck:
|
||||
test: mysql -D db -u user --password=password -e "select 1;"
|
||||
interval: 3s
|
||||
timeout: 2s
|
||||
retries: 40
|
||||
start_period: 2s
|
52
tests/testflows/extended_precision_data_types/regression.py
Executable file
52
tests/testflows/extended_precision_data_types/regression.py
Executable file
@ -0,0 +1,52 @@
|
||||
#!/usr/bin/env python3
|
||||
import os
|
||||
import sys
|
||||
|
||||
from testflows.core import *
|
||||
|
||||
append_path(sys.path, "..")
|
||||
|
||||
from helpers.cluster import Cluster
|
||||
from helpers.argparser import argparser
|
||||
from extended_precision_data_types.requirements import *
|
||||
|
||||
xfails = {
|
||||
}
|
||||
|
||||
xflags = {
|
||||
}
|
||||
|
||||
@TestModule
|
||||
@ArgumentParser(argparser)
|
||||
@XFails(xfails)
|
||||
@XFlags(xflags)
|
||||
@Name("extended precision data types")
|
||||
@Specifications(
|
||||
QA_SRS020_ClickHouse_Extended_Precision_Data_Types
|
||||
)
|
||||
@Requirements(
|
||||
RQ_SRS_020_ClickHouse_Extended_Precision("1.0"),
|
||||
)
|
||||
def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
|
||||
"""Extended precision data type regression.
|
||||
"""
|
||||
|
||||
top().terminating = False
|
||||
|
||||
nodes = {
|
||||
"clickhouse":
|
||||
("clickhouse1",)
|
||||
}
|
||||
with Cluster(local, clickhouse_binary_path, nodes=nodes,
|
||||
docker_compose_project_dir=os.path.join(current_dir(), "extended-precision-data-type_env")) as cluster:
|
||||
|
||||
self.context.cluster = cluster
|
||||
self.context.stress = stress
|
||||
|
||||
if parallel is not None:
|
||||
self.context.parallel = parallel
|
||||
|
||||
Feature(run=load("extended_precision_data_types.tests.feature", "feature"))
|
||||
|
||||
if main():
|
||||
regression()
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user