Merge branch 'master' into i24461

This commit is contained in:
mergify[bot] 2021-05-26 23:37:20 +00:00 committed by GitHub
commit d813d03a6a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
75 changed files with 7041 additions and 966 deletions

View File

@ -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.

View File

@ -80,7 +80,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/

View File

@ -199,7 +199,7 @@ Provides possibility to start background move data according to [TTL table expre
Returns `Ok.` even if table doesnt exist. Returns error when database doesnt 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}

View File

@ -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})

View File

@ -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
View 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
View 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;
};
}

View File

@ -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
View 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
View 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;
};
}

View 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

View 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

View File

@ -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");
};
}

View File

@ -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

View 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

View 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

View File

@ -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
}
}

View File

@ -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();

View File

@ -1,4 +1,5 @@
#pragma once
#include <Disks/DiskLocal.h>
#include <Disks/DiskMemory.h>
#include <Disks/IDisk.h>

View 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

View File

@ -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()

View File

@ -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)

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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)
{

View File

@ -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>

View File

@ -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

View File

@ -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;
};

View File

@ -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;

View File

@ -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_))
{
}

View File

@ -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

View File

@ -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);

View File

@ -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>

View File

@ -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"

View 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>

View File

@ -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>

View 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")

View 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>

View File

@ -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>

View 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

View File

@ -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

View File

@ -0,0 +1,4 @@
1 2020-11-11
1 2021-01-01
1 2020-11-11
1 2021-01-01

View 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;

View File

@ -236,4 +236,5 @@
01801_s3_distributed
01833_test_collation_alvarotuso
01850_dist_INSERT_preserve_error
01870_modulo_partition_key
01880_remote_ipv6

View 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

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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>

View File

@ -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 &lt; 1 or c - d &gt; 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>

View 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')

View File

@ -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

View File

@ -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

View File

@ -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

View 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()

View File

@ -0,0 +1 @@
from .requirements import *

View File

@ -0,0 +1,565 @@
# QA-SRS020 ClickHouse Extended Precision Data Types
# Software Requirements Specification
## Table of Contents
* 1 [Revision History](#revision-history)
* 2 [Introduction](#introduction)
* 3 [Terminology](#terminology)
* 3.1 [Extended Precision Data Types](#extended-precision-data-types)
* 4 [Requirements](#requirements)
* 4.1 [RQ.SRS-020.ClickHouse.Extended.Precision](#rqsrs-020clickhouseextendedprecision)
* 4.2 [Conversion](#conversion)
* 4.2.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt128](#rqsrs-020clickhouseextendedprecisionconversiontoint128)
* 4.2.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt128](#rqsrs-020clickhouseextendedprecisionconversiontouint128)
* 4.2.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt256](#rqsrs-020clickhouseextendedprecisionconversiontoint256)
* 4.2.4 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt256](#rqsrs-020clickhouseextendedprecisionconversiontouint256)
* 4.2.5 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toDecimal256](#rqsrs-020clickhouseextendedprecisionconversiontodecimal256)
* 4.2.6 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.FromMySQL](#rqsrs-020clickhouseextendedprecisionconversionfrommysql)
* 4.2.7 [RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.ToMySQL](#rqsrs-020clickhouseextendedprecisionconversiontomysql)
* 4.3 [Arithmetic](#arithmetic)
* 4.3.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Int.Supported](#rqsrs-020clickhouseextendedprecisionarithmeticintsupported)
* 4.3.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.Supported](#rqsrs-020clickhouseextendedprecisionarithmeticdecsupported)
* 4.3.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionarithmeticdecnotsupported)
* 4.4 [Arrays](#arrays)
* 4.4.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.Supported](#rqsrs-020clickhouseextendedprecisionarraysintsupported)
* 4.4.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.NotSupported](#rqsrs-020clickhouseextendedprecisionarraysintnotsupported)
* 4.4.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.Supported](#rqsrs-020clickhouseextendedprecisionarraysdecsupported)
* 4.4.4 [RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionarraysdecnotsupported)
* 4.5 [Comparison](#comparison)
* 4.5.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Comparison](#rqsrs-020clickhouseextendedprecisioncomparison)
* 4.6 [Logical Functions](#logical-functions)
* 4.6.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Logical](#rqsrs-020clickhouseextendedprecisionlogical)
* 4.7 [Mathematical Functions](#mathematical-functions)
* 4.7.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.Supported](#rqsrs-020clickhouseextendedprecisionmathematicalsupported)
* 4.7.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.NotSupported](#rqsrs-020clickhouseextendedprecisionmathematicalnotsupported)
* 4.8 [Rounding Functions](#rounding-functions)
* 4.8.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.Supported](#rqsrs-020clickhouseextendedprecisionroundingintsupported)
* 4.8.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.NotSupported](#rqsrs-020clickhouseextendedprecisionroundingintnotsupported)
* 4.8.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.Supported](#rqsrs-020clickhouseextendedprecisionroundingdecsupported)
* 4.8.4 [RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionroundingdecnotsupported)
* 4.9 [Bit Functions](#bit-functions)
* 4.9.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.Supported](#rqsrs-020clickhouseextendedprecisionbitintsupported)
* 4.9.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.NotSupported](#rqsrs-020clickhouseextendedprecisionbitintnotsupported)
* 4.9.3 [RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Dec.NotSupported](#rqsrs-020clickhouseextendedprecisionbitdecnotsupported)
* 4.10 [Null Functions](#null-functions)
* 4.10.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Null](#rqsrs-020clickhouseextendedprecisionnull)
* 4.11 [Tuple Functions](#tuple-functions)
* 4.11.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Tuple](#rqsrs-020clickhouseextendedprecisiontuple)
* 4.12 [Map Functions](#map-functions)
* 4.12.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Map.Supported](#rqsrs-020clickhouseextendedprecisionmapsupported)
* 4.12.2 [RQ.SRS-020.ClickHouse.Extended.Precision.Map.NotSupported](#rqsrs-020clickhouseextendedprecisionmapnotsupported)
* 4.13 [Create](#create)
* 4.13.1 [RQ.SRS-020.ClickHouse.Extended.Precision.Create.Table](#rqsrs-020clickhouseextendedprecisioncreatetable)
* 5 [References](#references)
## Revision History
This document is stored in an electronic form using [Git] source control management software
hosted in a [GitHub Repository].
All the updates are tracked using the [Revision History].
## Introduction
This software requirements specification covers requirements related to [ClickHouse]
using extended precision data types.
## Terminology
### Extended Precision Data Types
Inclusive bounds:
* Int128 - [-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727]
* UInt128 - [0 : 340282366920938463463374607431768211455]
* Int256 - [-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967]
* UInt256 - [0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935]
Exclusive bounds:
* Decimal256 - (10^(76 - S): 10^(76 - S)), where S is the scale.
## Requirements
### RQ.SRS-020.ClickHouse.Extended.Precision
version: 1.0
[ClickHouse] SHALL support using [Extended Precision Data Types].
### Conversion
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt128
version: 1.0
[ClickHouse] SHALL support converting values to `Int128` using the `toInt128` function.
For example,
```sql
SELECT toInt128(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt128
version: 1.0
[ClickHouse] SHALL support converting values to `UInt128` format using `toUInt128` function.
For example,
```sql
SELECT toUInt128(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toInt256
version: 1.0
[ClickHouse] SHALL support converting values to `Int256` using `toInt256` function.
For example,
```sql
SELECT toInt256(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toUInt256
version: 1.0
[ClickHouse] SHALL support converting values to `UInt256` format using `toUInt256` function.
For example,
```sql
SELECT toUInt256(1)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.toDecimal256
version: 1.0
[ClickHouse] SHALL support converting values to `Decimal256` format using `toDecimal256` function.
For example,
```sql
SELECT toDecimal256(1,2)
```
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.FromMySQL
version: 1.0
[ClickHouse] SHALL support converting to [Extended Precision Data Types] from MySQL.
#### RQ.SRS-020.ClickHouse.Extended.Precision.Conversion.ToMySQL
version: 1.0
[ClickHouse] MAY not support converting from [Extended Precision Data Types] to MySQL.
### Arithmetic
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Int.Supported
version: 1.0
[ClickHouse] SHALL support using [Arithmetic functions] with Int128, UInt128, Int256, and UInt256.
Arithmetic functions:
* plus
* minus
* multiply
* divide
* intDiv
* intDivOrZero
* modulo
* moduloOrZero
* negate
* abs
* gcd
* lcm
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Arithmetic functions] with Decimal256:
* plus
* minus
* multiply
* divide
* intDiv
* intDivOrZero
* negate
* abs
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arithmetic.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Arithmetic functions] with Decimal256:
* modulo
* moduloOrZero
* gcd
* lcm
### Arrays
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Array functions] with Int128, UInt128, Int256, and UInt256.
* empty
* notEmpty
* length
* arrayCount
* arrayPopBack
* arrayPopFront
* arraySort
* arrayReverseSort
* arrayUniq
* arrayJoin
* arrayDistinct
* arrayEnumerate
* arrayEnumerateDense
* arrayEnumerateUniq
* arrayReverse
* reverse
* arrayFlatten
* arrayCompact
* arrayExists
* arrayAll
* arrayMin
* arrayMax
* arraySum
* arrayAvg
* arrayReduce
* arrayReduceInRanges
* arrayZip
* arrayMap
* arrayFilter
* arrayFill
* arrayReverseFill
* arraySplit
* arrayFirst
* arrayFirstIndex
* arrayConcat
* hasAll
* hasAny
* hasSubstr
* arrayElement
* has
* indexOf
* countEqual
* arrayPushBack
* arrayPushFront
* arrayResize
* arraySlice
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Int.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Array functions] with Int128, UInt128, Int256, and UInt256:
* arrayDifference
* arrayCumSum
* arrayCumSumNonNegative
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Array functions] with Decimal256:
* empty
* notEmpty
* length
* arrayCount
* arrayPopBack
* arrayPopFront
* arraySort
* arrayReverseSort
* arrayUniq
* arrayJoin
* arrayDistinct
* arrayEnumerate
* arrayEnumerateDense
* arrayEnumerateUniq
* arrayReverse
* reverse
* arrayFlatten
* arrayCompact
* arrayExists
* arrayAll
* arrayReduce
* arrayReduceInRanges
* arrayZip
* arrayMap
* arrayFilter
* arrayFill
* arrayReverseFill
* arraySplit
* arrayFirst
* arrayFirstIndex
* arrayConcat
* hasAll
* hasAny
* hasSubstr
* arrayElement
* has
* indexOf
* countEqual
* arrayPushBack
* arrayPushFront
* arrayResize
* arraySlice
#### RQ.SRS-020.ClickHouse.Extended.Precision.Arrays.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Array functions] with Decimal256:
* arrayMin
* arrayMax
* arraaySum
* arrayAvg
* arrayDifference
* arrayCumSum
* arrayCumSumNonNegative
### Comparison
#### RQ.SRS-020.ClickHouse.Extended.Precision.Comparison
version: 1.0
[ClickHouse] SHALL support using [Comparison functions] with [Extended Precision Data Types].
Comparison functions:
* equals
* notEquals
* less
* greater
* lessOrEquals
* greaterOrEquals
### Logical Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Logical
version: 1.0
[ClickHouse] MAY not support using [Logical functions] with [Extended Precision Data Types].
Logical functions:
* and
* or
* not
* xor
### Mathematical Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Mathematical functions] with [Extended Precision Data Types]:
* exp
* log, ln
* exp2
* log2
* exp10
* log10
* sqrt
* cbrt
* erf
* erfc
* lgamma
* tgamma
* sin
* cos
* tan
* asin
* acos
* atan
* cosh
* acosh
* sinh
* asinh
* tanh
* atanh
* log1p
* sign
#### RQ.SRS-020.ClickHouse.Extended.Precision.Mathematical.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Mathematical functions] with [Extended Precision Data Types]:
* pow, power
* intExp2
* intExp10
* atan2
* hypot
### Rounding Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Rounding functions] with Int128, UInt128, Int256, and UInt256:
* floor
* ceil
* trunc
* round
* roundBankers
* roundDuration
* roundAge
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Int.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Rounding functions] with Int128, UInt128, Int256, and UInt256:
* roundDown
* roundToExp2
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Rounding functions] with Decimal256:
* floor
* ceil
* trunc
* round
* roundBankers
#### RQ.SRS-020.ClickHouse.Extended.Precision.Rounding.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Rounding functions] with Decimal256:
* roundDuration
* roundAge
* roundDown
* roundToExp2
### Bit Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Bit functions] with Int128, UInt128, Int256, and UInt256:
* bitAnd
* bitOr
* bitXor
* bitNot
* bitShiftLeft
* bitShiftRight
* bitCount
#### RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Int.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Bit functions] with Int128, UInt128, Int256, and UInt256:
* bitRotateLeft
* bitRotateRight
* bitTest
* bitTestAll
* bitTestAny
#### RQ.SRS-020.ClickHouse.Extended.Precision.Bit.Dec.NotSupported
version: 1.0
[ClickHouse] MAY not support using [Bit functions] with Decimal256.
Bit functions:
* bitAnd
* bitOr
* bitXor
* bitNot
* bitShiftLeft
* bitShiftRight
* bitCount
* bitRotateLeft
* bitRotateRight
* bitTest
* bitTestAll
* bitTestAny
### Null Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Null
version: 1.0
[ClickHouse] SHALL support using [Null functions] with [Extended Precision Data Types].
Null functions:
* isNull
* isNotNull
* coalesce
* ifNull
* nullIf
* assumeNotNull
* toNullable
### Tuple Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Tuple
version: 1.0
[ClickHouse] SHALL support using [Tuple functions] with [Extended Precision Data Types].
Tuple functions:
* tuple
* tupleElement
* untuple
### Map Functions
#### RQ.SRS-020.ClickHouse.Extended.Precision.Map.Supported
version: 1.0
[ClickHouse] SHALL support using the following [Map functions] with [Extended Precision Data Types]:
* map
* mapContains
* mapKeys
* mapValues
#### RQ.SRS-020.ClickHouse.Extended.Precision.Map.NotSupported
version: 1.0
[ClickHouse] MAY not support using the following [Map functions] with [Extended Precision Data Types]:
* mapAdd
* mapSubtract
* mapPopulateSeries
### Create
#### RQ.SRS-020.ClickHouse.Extended.Precision.Create.Table
version: 1.0
[ClickHouse] SHALL support creating table with columns that use [Extended Precision Data Types].
## References
* **ClickHouse:** https://clickhouse.tech
* **GitHub Repository**: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
* **Revision History**: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
* **Git:** https://git-scm.com/
[Extended Precision Data Types]: #extended-precision-data-types
[Arithmetic functions]: https://clickhouse.tech/docs/en/sql-reference/functions/arithmetic-functions/
[Array functions]: https://clickhouse.tech/docs/en/sql-reference/functions/array-functions/
[Comparison functions]: https://clickhouse.tech/docs/en/sql-reference/functions/comparison-functions/
[Logical Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/logical-functions/
[Mathematical Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/math-functions/
[Rounding Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/rounding-functions/
[Bit Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/bit-functions/
[Null Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/functions-for-nulls/
[Tuple Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/tuple-functions/
[Map Functions]: https://clickhouse.tech/docs/en/sql-reference/functions/tuple-map-functions/
[SRS]: #srs
[ClickHouse]: https://clickhouse.tech
[GitHub Repository]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
[Revision History]: https://github.com/ClickHouse/ClickHouse/blob/master/tests/testflows/extended_precision_data_types/requirements/requirements.md
[Git]: https://git-scm.com/
[GitHub]: https://github.com

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,217 @@
import os
import textwrap
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('plus', '2'),
('minus', '0'),
('multiply', '1'),
('divide', '1'),
('intDiv', '1'),
('intDivOrZero', '1'),
('modulo', '0'),
('moduloOrZero', '0'),
('negate', '-1'),
('abs', '1'),
('gcd', '1'),
('lcm', '1'),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline
@Examples('arithmetic_func expected_result int_type min max', Examples_list)
def inline_check(self, arithmetic_func, expected_result, int_type, min, max, node=None):
"""Check that arithmetic functions work using inline tests with Int128, UInt128, Int256, and UInt256.
"""
if node is None:
node = self.context.node
if arithmetic_func in ['negate','abs']:
with When(f"I check {arithmetic_func} with {int_type}"):
output = node.query(f"SELECT {arithmetic_func}(to{int_type}(1))").output
assert output == expected_result, error()
with When(f"I check {arithmetic_func} with {int_type} max and min value"):
execute_query(f"""
SELECT {arithmetic_func}(to{int_type}(\'{max}\')), {arithmetic_func}(to{int_type}(\'{min}\'))
""")
else:
with When(f"I check {arithmetic_func} with {int_type}"):
output = node.query(f"SELECT {arithmetic_func}(to{int_type}(1), to{int_type}(1))").output
assert output == expected_result, error()
if arithmetic_func in ['gcd','lcm']:
if int_type in ['UInt128','UInt256']:
exitcode=153
else:
exitcode=151
with When(f"I check {arithmetic_func} with {int_type} max and min value"):
node.query(f"SELECT {arithmetic_func}(to{int_type}(\'{max}\'), to{int_type}(1)), {arithmetic_func}(to{int_type}(\'{min}\'), to{int_type}(1))",
exitcode = exitcode, message = 'Exception:')
else:
with When(f"I check {arithmetic_func} with {int_type} max and min value"):
execute_query(f"""
SELECT {arithmetic_func}(to{int_type}(\'{max}\'), to{int_type}(1)), {arithmetic_func}(to{int_type}(\'{min}\'), to{int_type}(1))
""")
@TestOutline
@Examples('arithmetic_func expected_result int_type min max', Examples_list)
def table_check(self, arithmetic_func, expected_result, int_type, min, max, node=None):
"""Check that arithmetic functions work using tables with Int128, UInt128, Int256, and UInt256.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
if arithmetic_func in ['negate','abs']:
for value in [1, min, max]:
with When(f"I insert {arithmetic_func} with {int_type} {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{value}\'))")
with Then(f"I check the table output of {arithmetic_func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
with When(f"I insert {arithmetic_func} with {int_type} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(1), to{int_type}(1))")
with Then("I check that the output matches the expected value"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == expected_result, error()
if arithmetic_func in ['gcd', 'lcm']:
if int_type in ['UInt128', 'UInt256']:
with When(f"I insert {arithmetic_func} with {int_type} {min} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{min}\'), to{int_type}(1))",
exitcode = 153, message = 'Exception:')
with And(f"I insert {arithmetic_func} with {int_type} {max} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{max}\'), to{int_type}(1))")
else:
for value in [min, max]:
with When(f"I insert {arithmetic_func} with {int_type} {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{value}\'), to{int_type}(1))",
exitcode = 151, message = 'Exception:')
else:
for value in [min, max]:
with When(f"I insert {arithmetic_func} with {int_type} {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(to{int_type}(\'{value}\'), to{int_type}(1))")
with Then(f"I check the table output of {arithmetic_func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline
@Examples('arithmetic_func expected_result', Examples_dec_list)
def inline_check_dec(self, arithmetic_func, expected_result, node=None):
"""Check that arithmetic functions work using inline with Decimal256.
"""
if node is None:
node = self.context.node
if arithmetic_func is 'negate' or arithmetic_func is 'abs':
with When(f"I check {arithmetic_func} with toDecimal256"):
output = node.query(f"SELECT {arithmetic_func}(toDecimal256(1,0))").output
assert output == expected_result, error()
elif arithmetic_func in ['modulo', 'moduloOrZero', 'gcd', 'lcm']:
with When(f"I check {arithmetic_func} with toDecimal256"):
node.query(f"SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))",
exitcode=43, message = 'Exception:')
else:
with When(f"I check {arithmetic_func} with toDecimal256"):
output = node.query(f"SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))").output
assert output == expected_result, error()
@TestOutline
@Examples('arithmetic_func expected_result', Examples_dec_list)
def table_check_dec(self, arithmetic_func, expected_result, node=None):
"""Check that arithmetic functions work using tables with Decimal256.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if arithmetic_func in ['negate','abs']:
with When(f"I insert {arithmetic_func} with toDecimal256 into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(toDecimal256(1,0))")
with Then(f"I check the table for output of {arithmetic_func} with Decimal256"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
elif arithmetic_func in ['modulo', 'moduloOrZero', 'gcd', 'lcm']:
with When(f"I check {arithmetic_func} with toDecimal256"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))",
exitcode=43, message = 'Exception:')
else:
with When(f"I insert {arithmetic_func} with toDecimal256 into the table"):
node.query(f"INSERT INTO {table_name} SELECT {arithmetic_func}(toDecimal256(1,0), toDecimal256(1,0))")
with Then("I check that the output matches the expected value"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == expected_result, error()
@TestFeature
@Name("arithmetic")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Arithmetic_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arithmetic_Dec_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arithmetic_Dec_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that arithmetic functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run = inline_check)
Scenario(run = table_check)
Scenario(run = inline_check_dec)
Scenario(run = table_check_dec)

View File

@ -0,0 +1,484 @@
import uuid
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
def get_table_name():
return "table" + "_" + str(uuid.uuid1()).replace('-', '_')
@TestOutline(Suite)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Int_NotSupported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Dec_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Arrays_Dec_NotSupported("1.0"),
)
def array_func(self, data_type, node=None):
"""Check array functions with extended precision data types.
"""
if node is None:
node = self.context.node
for func in ['arrayPopBack(',
'arrayPopFront(',
'arraySort(',
'arrayReverseSort(',
'arrayDistinct(',
'arrayEnumerate(',
'arrayEnumerateDense(',
'arrayEnumerateUniq(',
'arrayReverse(',
'reverse(',
'arrayFlatten(',
'arrayCompact(',
'arrayReduceInRanges(\'sum\', [(1, 5)],',
'arrayMap(x -> (x + 2),',
'arrayFill(x -> x=3,',
'arrayReverseFill(x -> x=3,',
f'arrayConcat([{to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}],',
'arrayFilter(x -> x == 1, ']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arraySplit((x, y) -> x=y, [0, 0, 0],']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array(Array({data_type}))')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in [f'arrayZip([{to_data_type(data_type,1)}],']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array(Tuple({data_type}, {data_type}))')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['empty(',
'notEmpty(',
'length(',
'arrayCount(x -> x == 1, ',
'arrayUniq(',
'arrayJoin(',
'arrayExists(x -> x==1,',
'arrayAll(x -> x==1,',
'arrayMin(',
'arrayMax(',
'arraySum(',
'arrayAvg(',
'arrayReduce(\'max\', ',
'arrayFirst(x -> x==3,',
'arrayFirstIndex(x -> x==3,',
f'hasAll([{to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}], ',
f'hasAny([{to_data_type(data_type,2)}, {to_data_type(data_type,1)}], ',
f'hasSubstr([{to_data_type(data_type,2)}, {to_data_type(data_type,1)}], ']:
if func in ['arrayMin(','arrayMax(','arraySum(', 'arrayAvg('] and data_type in ['Decimal256(0)']:
with Scenario(f"Inline - {data_type} - {func}"):
node.query(f"SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = 44, message = 'Exception:')
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = 44, message = 'Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayDifference(',
'arrayCumSum(',
'arrayCumSumNonNegative(']:
if data_type in ['Decimal256(0)']:
exitcode = 44
else:
exitcode = 43
with Scenario(f"Inline - {data_type} - {func}"):
node.query(f"SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = exitcode, message = 'Exception:')
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}))",
exitcode = exitcode, message = 'Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayElement']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayPushBack',
'arrayPushFront']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), {to_data_type(data_type,1)})
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), {to_data_type(data_type,1)})")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['arrayResize',
'arraySlice']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), 1)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
for func in ['has',
'indexOf',
'countEqual']:
with Scenario(f"Inline - {data_type} - {func}"):
execute_query(f"""
SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), NULL)
""")
with Scenario(f"Table - {data_type} - {func}"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(array({to_data_type(data_type,3)}, {to_data_type(data_type,2)}, {to_data_type(data_type,1)}), NULL)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Suite)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Tuple("1.0"),
)
def tuple_func(self, data_type, node=None):
"""Check tuple functions with extended precision data types.
"""
if node is None:
node = self.context.node
with Scenario(f"Creating a tuple with {data_type}"):
node.query(f"SELECT tuple({to_data_type(data_type,1)}, {to_data_type(data_type,1)}, {to_data_type(data_type,1)})")
with Scenario(f"Creating a tuple with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple({data_type}, {data_type}, {data_type})')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT tuple({to_data_type(data_type,1)}, {to_data_type(data_type,1)}, {to_data_type(data_type,1)})")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"tupleElement with {data_type}"):
node.query(f"SELECT tupleElement(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), 1)")
with Scenario(f"tupleElement with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT tupleElement(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), 1)")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"untuple with {data_type}"):
node.query(f"SELECT untuple(({to_data_type(data_type,1)},))")
with Scenario(f"untuple with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT untuple(({to_data_type(data_type,1)},))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"tupleHammingDistance with {data_type}"):
node.query(f"SELECT tupleHammingDistance(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), ({to_data_type(data_type,2)}, {to_data_type(data_type,2)}))")
with Scenario(f"tupleHammingDistance with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT tupleHammingDistance(({to_data_type(data_type,1)}, {to_data_type(data_type,1)}), ({to_data_type(data_type,2)}, {to_data_type(data_type,2)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Suite)
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Map_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Map_NotSupported("1.0"),
)
def map_func(self, data_type, node=None):
"""Check Map functions with extended precision data types.
"""
if node is None:
node = self.context.node
with Scenario(f"Creating a map with {data_type}"):
node.query(f"SELECT map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)})")
with Scenario(f"Creating a map with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Map(String, {data_type})')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)})")
execute_query(f"""
SELECT * FROM {table_name}
""")
with Scenario(f"mapAdd with {data_type}"):
node.query(f"SELECT mapAdd(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
with Scenario(f"mapAdd with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapAdd(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapSubtract with {data_type}"):
node.query(f"SELECT mapSubtract(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
with Scenario(f"mapSubtract with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapSubtract(([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]), ([{to_data_type(data_type,1)}, {to_data_type(data_type,2)}], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}]))",
exitcode = 44, message='Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapPopulateSeries with {data_type}"):
node.query(f"SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)",
exitcode = 44, message='Exception:')
with Scenario(f"mapPopulateSeries with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Tuple(Array({data_type}), Array({data_type}))')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapPopulateSeries([1,2,3], [{to_data_type(data_type,1)}, {to_data_type(data_type,2)}, {to_data_type(data_type,3)}], 5)",
exitcode = 44, message='Exception:')
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapContains with {data_type}"):
node.query(f"SELECT mapContains( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}), 'key1')")
with Scenario(f"mapContains with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = data_type)
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapContains( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}), 'key1')")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapKeys with {data_type}"):
node.query(f"SELECT mapKeys( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
with Scenario(f"mapKeys with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = 'Array(String)')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapKeys( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
with Scenario(f"mapValues with {data_type}"):
node.query(f"SELECT mapValues( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
with Scenario(f"mapValues with {data_type} on a table"):
table_name = get_table_name()
table(name = table_name, data_type = f'Array({data_type})')
with When("I insert the output into a table"):
node.query(f"INSERT INTO {table_name} SELECT mapValues( map('key1', {to_data_type(data_type,1)}, 'key2', {to_data_type(data_type,2)}))")
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("array, tuple, map")
@Examples("data_type",[
('Int128',),
('Int256',),
('UInt128',),
('UInt256',),
('Decimal256(0)',),
])
def feature(self, node="clickhouse1", stress=None, parallel=None):
"""Check that array, tuple, and map functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
with allow_experimental_bigint(self.context.node):
for example in self.examples:
data_type, = example
with Feature(data_type):
Suite(test=array_func)(data_type=data_type)
Suite(test=tuple_func)(data_type=data_type)
with Given("I allow experimental map type"):
allow_experimental_map_type()
Suite(test=map_func)(data_type=data_type)

View File

@ -0,0 +1,179 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
from extended_precision_data_types.errors import *
funcs = [
('bitAnd', True, None),
('bitOr', True, None),
('bitXor', True, None),
('bitShiftLeft', True, None),
('bitShiftRight', True, None),
('bitRotateLeft', False, not_implemented_bigints('Bit rotate')),
('bitRotateRight', False, not_implemented_bigints('Bit rotate')),
('bitTest', False, not_implemented_bigints('bitTest')),
('bitTestAll', False, illegal_column()),
('bitTestAny', False, illegal_column()),
('bitNot', True, None),
('bitCount', True, None)
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline(Scenario)
@Examples('func supported error int_type min max', Examples_list)
def bit_int_inline(self, func, supported, error, int_type, min, max, node=None):
""" Check bit functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if error is not None:
exitcode,message = error
if node is None:
node = self.context.node
if func in ["bitNot", "bitCount"]:
with When(f"Check {func} with {int_type}"):
execute_query(f"""
SELECT {func}(to{int_type}(1)), {func}(to{int_type}(\'{max}\')), {func}(to{int_type}(\'{min}\'))
""")
elif supported:
with When(f"I check {func} with {int_type}"):
execute_query(f"""
SELECT {func}(to{int_type}(1), 1), {func}(to{int_type}(\'{max}\'), 1), {func}(to{int_type}(\'{min}\'), 1)
""")
else:
with When(f"I check {func} with {int_type}"):
node.query(f"SELECT {func}(to{int_type}(1), 1), {func}(to{int_type}(\'{max}\'), 1), {func}(to{int_type}(\'{min}\'), 1)",
exitcode=exitcode, message = message)
@TestOutline(Scenario)
@Examples('func supported error int_type min max', Examples_list)
def bit_int_table(self, func, supported, error, int_type, min, max, node=None):
""" Check bit functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
if error is not None:
exitcode,message = error
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
if func in ["bitNot", "bitCount"]:
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'))")
with Then(f"I check the table with values of {func} and {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
elif supported:
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), 1)")
with Then(f"I check the table with values of {func} and {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), 1)",
exitcode=exitcode, message=message)
@TestOutline(Scenario)
@Examples('func supported error', Examples_dec_list)
def bit_dec_inline(self, func, supported, error, node=None):
""" Check bit functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
exitcode, message = illegal_type()
if node is None:
node = self.context.node
if func in ["bitNot", "bitCount"]:
with When(f"Check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0)), {func}(toDecimal256(\'{min}\',0))",
exitcode=exitcode, message = message)
else:
with When(f"I check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0), 1), {func}(toDecimal256(\'{max}\',0), 1), {func}(toDecimal256(\'{min}\',0), 1)",
exitcode=exitcode, message = message)
@TestOutline(Scenario)
@Examples('func supported error', Examples_dec_list)
def bit_dec_table(self, func, supported, error, node=None):
""" Check bit functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
table_name = f"table_{getuid()}"
exitcode, message = illegal_type()
if node is None:
node = self.context.node
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if func in ["bitNot", "bitCount"]:
for value in [1, min, max]:
with When(f"I insert the output of {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0))",
exitcode=exitcode, message = message)
else:
for value in [1, min, max]:
with When(f"I insert the output of {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0), 1)",
exitcode=exitcode, message=message)
@TestFeature
@Name("bit")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Bit_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Bit_Int_NotSupported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Bit_Dec_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that bit functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=bit_int_inline)
Scenario(run=bit_int_table)
Scenario(run=bit_dec_inline)
Scenario(run=bit_dec_table)

View File

@ -0,0 +1,110 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('equals',),
('notEquals',),
('less',),
('greater',),
('lessOrEquals',),
('greaterOrEquals',)
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def comp_int_inline(self, func, int_type, min, max, node=None):
"""Check comparison functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
with When(f"I check {func} with {int_type}"):
execute_query(f"""
SELECT {func}(to{int_type}(1), to{int_type}(1)), {func}(to{int_type}(\'{max}\'), to{int_type}(\'{min}\'))
""")
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def comp_int_table(self, func, int_type, min, max, node=None):
"""Check comparison functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
for value in [1, max, min]:
with When(f"I insert into a table the output {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), to{int_type}(1))")
with Then(f"I check the table for the output of {func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Scenario)
@Examples('func', funcs)
def comp_dec_inline(self, func, node=None):
"""Check comparison functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
with When(f"I check {func} with Decimal256"):
execute_query(f"""
SELECT {func}(toDecimal256(1,0), toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0), toDecimal256(\'{min}\',0))
""")
@TestOutline(Scenario)
@Examples('func', funcs)
def comp_dec_table(self, func, node=None):
"""Check comparison functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
for value in [1, max, min]:
with When(f"I insert into a table the output {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0), toDecimal256(1,0))")
with Then(f"I check the table for the output of {func} with Decimal256"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("comparison")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Comparison("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that comparison functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=comp_int_inline)
Scenario(run=comp_int_table)
Scenario(run=comp_dec_inline)
Scenario(run=comp_dec_table)

View File

@ -0,0 +1,275 @@
import os
import textwrap
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
@contextmanager
def dictionary(name, node, mysql_node):
"""Create a table in MySQL and use it a source for a dictionary.
"""
try:
with Given("table in MySQL"):
sql = f"""
CREATE TABLE {name}(
id INT NOT NULL AUTO_INCREMENT,
int128 BIGINT,
uint128 BIGINT,
int256 BIGINT,
uint256 BIGINT,
dec256 DECIMAL,
PRIMARY KEY ( id )
);
"""
with When("I drop the table if exists"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
with And("I create a table"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with And("dictionary that uses MySQL table as the external source"):
with When("I drop the dictionary if exists"):
node.query(f"DROP DICTIONARY IF EXISTS dict_{name}")
with And("I create the dictionary"):
sql = f"""
CREATE DICTIONARY dict_{name}
(
id UInt8,
int128 Int128,
uint128 UInt128,
int256 Int256,
uint256 UInt256,
dec256 Decimal256(0)
)
PRIMARY KEY id
SOURCE(MYSQL(
USER 'user'
PASSWORD 'password'
DB 'db'
TABLE '{name}'
REPLICA(PRIORITY 1 HOST '{mysql_node.name}' PORT 3306)
))
LAYOUT(HASHED())
LIFETIME(0)
"""
node.query(textwrap.dedent(sql))
yield f"dict_{name}"
finally:
with Finally("I drop the dictionary", flags=TE):
node.query(f"DROP DICTIONARY IF EXISTS dict_{name}")
with And("I drop a table in MySQL", flags=TE):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
@contextmanager
def table(name, node, mysql_node):
"""Create a table in MySQL and use it a source for a table in ClickHouse.
"""
try:
with Given("table in MySQL"):
sql = f"""
CREATE TABLE {name}(
id INT NOT NULL AUTO_INCREMENT,
int128 BIGINT,
uint128 BIGINT,
int256 BIGINT,
uint256 BIGINT,
dec256 DECIMAL,
PRIMARY KEY ( id )
);
"""
with When("I drop the table if exists"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
with And("I create a table"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with And("table that uses MySQL table as the external source"):
with When("I drop the table if exists"):
node.query(f"DROP TABLE IF EXISTS {name}")
with And("I create the table"):
sql = f"""
CREATE TABLE {name}
(
id UInt8,
int128 Int128,
uint128 UInt128,
int256 Int256,
uint256 UInt256,
dec256 Decimal256(0)
)
ENGINE = MySQL('{mysql_node.name}:3306', 'default', '{name}', 'default', 'password')
"""
node.query(textwrap.dedent(sql))
yield f"table_{name}"
finally:
with Finally("I drop the table", flags=TE):
node.query(f"DROP TABLE IF EXISTS {name}")
with And("I drop a table in MySQL", flags=TE):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
@contextmanager
def table_func(name, node, mysql_node):
"""Create a table in MySQL and use it a source for a table using mysql table function.
"""
try:
with Given("table in MySQL"):
sql = f"""
CREATE TABLE {name}(
id INT NOT NULL AUTO_INCREMENT,
int128 BIGINT,
uint128 BIGINT,
int256 BIGINT,
uint256 BIGINT,
dec256 DECIMAL,
PRIMARY KEY ( id )
);
"""
with When("I drop the table if exists"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
with And("I create a table"):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
yield f"mysql('{mysql_node.name}:3306', 'db', '{name}', 'user', 'password')"
finally:
with Finally("I drop the table", flags=TE):
node.query(f"DROP TABLE IF EXISTS {name}")
with And("I drop a table in MySQL", flags=TE):
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user -e \"DROP TABLE IF EXISTS {name};\"", exitcode=0)
@TestOutline(Scenario)
@Examples('int_type min max',[
('Int128', '-170141183460469231731687303715884105728', '170141183460469231731687303715884105727', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toInt128("1.0")), Name('Int128')),
('Int256', '-57896044618658097711785492504343953926634992332820282019728792003956564819968', '57896044618658097711785492504343953926634992332820282019728792003956564819967', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toInt256("1.0")), Name('Int256')),
('UInt128','0','340282366920938463463374607431768211455', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toUInt128("1.0")), Name('UInt128')),
('UInt256', '0', '115792089237316195423570985008687907853269984665640564039457584007913129639935', Requirements(RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toUInt256("1.0")), Name('UInt256')),
])
def int_conversion(self, int_type, min, max, node=None):
"""Check that ClickHouse converts values to Int128.
"""
if node is None:
node = self.context.node
with When(f"I convert {min}, {max}, 1 to {int_type}"):
output = node.query(f"SELECT to{int_type}(\'{min}\'), to{int_type}(\'{max}\'), to{int_type}(1) format TabSeparatedRaw").output
assert output == f'{min}\t{max}\t1', error()
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_toDecimal256("1.0"),
)
def to_decimal256(self, node=None):
"""Check that ClickHouse converts values to Int128.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
for value in [1,min,max]:
output = node.query(f"SELECT toDecimal256(\'{value}\',0)").output
assert output == str(value), error()
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_ToMySQL("1.0"),
)
def MySQL_table(self, node=None):
"""Check that ClickHouse converts MySQL values from MySQL table into ClickHouse table.
"""
table_name = f'table_{getuid()}'
node = self.context.node
mysql_node = self.context.mysql_node
with table(table_name, node, mysql_node):
with When("I insert parameters values in MySQL"):
sql = f"""
INSERT INTO {table_name}(int128, uint128, int256, uint256, dec256) VALUES (1,1,1,1,1);
"""
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with Then("I select from the table on top of the mysql table"):
node.query(f"SELECT * FROM {table_name}",
exitcode=50, message='Exception:')
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_FromMySQL("1.0"),
)
def MySQL_func(self, node=None):
"""Check that ClickHouse converts MySQL values into a ClickHouse table using the MySQL table function.
"""
table_name = f'table_{getuid()}'
node = self.context.node
mysql_node = self.context.mysql_node
with table_func(table_name, node, mysql_node) as table_function:
with When("I insert parameters values in MySQL"):
sql = f"""
INSERT INTO {table_name}(int128, uint128, int256, uint256, dec256) VALUES (1,1,1,1,1);
"""
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with And("I make sure the table doesn't exist"):
node.query(f"DROP TABLE IF EXISTS {table_name}")
with And("I create the table"):
node.query(f"CREATE TABLE {table_name} (id UInt8, int128 Int128, uint128 UInt128, int256 Int256, uint256 UInt256, dec256 Decimal256(0)) Engine = Memory")
with And("I insert into the clickhouse table from the mysql table"):
node.query(f"INSERT INTO {table_name} SELECT * FROM {table_function}")
with Then("I select from the clickhouse table"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == '1\t1\t1\t1\t1\t1', error()
@TestScenario
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Conversion_ToMySQL("1.0"),
)
def MySQL_dict(self, node=None):
"""Check that ClickHouse converts MySQL values from MySQL table into ClickHouse dictionary.
"""
node = self.context.node
mysql_node = self.context.mysql_node
table_name = f'table_{getuid()}'
with dictionary(table_name, node, mysql_node):
with When("I insert parameters values in MySQL"):
sql = f"""
INSERT INTO {table_name}(int128, uint128, int256, uint256, dec256) VALUES (1,1,1,1,1);
"""
mysql_node.command(f"MYSQL_PWD=password mysql -D db -u user <<'EOF'{textwrap.dedent(sql)}\nEOF", exitcode=0)
with Then("I select from the table on top of the mysql table"):
node.query(f"SELECT * FROM dict_{table_name}",
exitcode=50, message='Exception:')
@TestFeature
@Name("conversion")
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check the conversion of extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
for scenario in loads(current_module(), Scenario):
with allow_experimental_bigint(self.context.node):
Scenario(run=scenario)

View File

@ -0,0 +1,19 @@
from testflows.core import *
from testflows.core.name import basename, parentname
from testflows._core.testtype import TestSubType
@TestFeature
@Name("tests")
def feature(self):
"""Check functions with Int128, Int256, UInt256, and Decimal256.
"""
Feature(run=load("extended_precision_data_types.tests.conversion", "feature"))
Feature(run=load("extended_precision_data_types.tests.arithmetic", "feature"))
Feature(run=load("extended_precision_data_types.tests.array_tuple_map", "feature"))
Feature(run=load("extended_precision_data_types.tests.comparison", "feature"))
Feature(run=load("extended_precision_data_types.tests.logical", "feature"))
Feature(run=load("extended_precision_data_types.tests.mathematical", "feature"))
Feature(run=load("extended_precision_data_types.tests.rounding", "feature"))
Feature(run=load("extended_precision_data_types.tests.bit", "feature"))
Feature(run=load("extended_precision_data_types.tests.null", "feature"))
Feature(run=load("extended_precision_data_types.tests.table", "feature"))

View File

@ -0,0 +1,99 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('and',),
('or',),
('not',),
('xor',),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def log_int_inline(self, func, int_type, min, max, node=None):
"""Check logical functions with Int128, Int256, and UInt256 using inline tests.
"""
table_name = f'table_{getuid()}'
if node is None:
node = self.context.node
with When(f"Check {func} with {int_type}"):
node.query(f"SELECT {func}(to{int_type}(1), to{int_type}(1)), {func}(to{int_type}(\'{max}\'), to{int_type}(1)), {func}(to{int_type}(\'{min}\'), to{int_type}(1))",
exitcode=43, message = 'Exception: Illegal type ')
@TestOutline(Scenario)
@Examples('func int_type min max', Examples_list)
def log_int_table(self, func, int_type, min, max, node=None):
"""Check logical functions with Int128, Int256, and UInt256 using table tests.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = int_type)
for value in [1, min, max]:
with When(f"Check {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'), to{int_type}(\'{value}\'))",
exitcode=43, message = 'Exception: Illegal type')
@TestOutline(Scenario)
@Examples('func', funcs)
def log_dec_inline(self, func, node=None):
"""Check logical functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
with When(f"Check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0), toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0), toDecimal256(1)), {func}(toDecimal256(\'{min}\',0), toDecimal256(1))",
exitcode=43, message = 'Exception: Illegal type ')
@TestOutline(Scenario)
@Examples('func', funcs)
def log_dec_table(self, func, node=None):
"""Check logical functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
for value in [1, min, max]:
with When(f"Check {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0), toDecimal256(\'{value}\',0))",
exitcode=43, message = 'Exception: Illegal type ')
@TestFeature
@Name("logical")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Logical("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that comparison functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=log_int_inline)
Scenario(run=log_int_table)
Scenario(run=log_dec_inline)
Scenario(run=log_dec_table)

View File

@ -0,0 +1,187 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('exp(', 3, 0),
('log(', 0, 0),
('ln(', 0, 0),
('exp2(', 2, 0),
('log2(', 0, 0),
('exp10(', 10, 0),
('log10(', 0, 0),
('sqrt(', 1, 0),
('cbrt(', 1, 0),
('erf(', 1, 0),
('erfc(', 0, 0),
('lgamma(', 0, 0),
('tgamma(', 1, 0),
('sin(', 1, 0),
('cos(', 1, 0),
('tan(', 2, 0),
('asin(', 2, 0),
('acos(', 0, 0),
('atan(', 1, 0),
('intExp2(', 2, 48),
('intExp10(', 10, 48),
('cosh(', 2, 0),
('acosh(', 0, 0),
('sinh(', 1, 0),
('asinh(', 1, 0),
('tanh(', 1, 0),
('atanh(', 'inf', 0),
('log1p(', 1, 0),
('sign(', 1, 0),
('pow(1,', 1, 43),
('power(1,', 1, 43),
('atan2(1,', 1, 43),
('hypot(1,', 1, 43),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline(Scenario)
@Examples('func expected_result exitcode int_type min max', Examples_list)
def math_int_inline(self, func, expected_result, exitcode, int_type, min, max, node=None):
"""Check mathematical functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
with When(f"I check {func} with {int_type} using 1, max, and min"):
node.query(f"SELECT {func} to{int_type}(1)), {func} to{int_type}(\'{max}\')), {func} to{int_type}(\'{min}\'))",
exitcode=exitcode, message = 'Exception:')
else:
with When(f"I check {func} with {int_type} using 1"):
output = node.query(f"SELECT {func} to{int_type}(1))").output
if output == 'inf':
pass
else:
assert round(float(output)) == expected_result, error()
with And(f"I check {func} with {int_type} using max and min"):
execute_query(f"""
SELECT {func} to{int_type}(\'{max}\')), {func} to{int_type}(\'{min}\'))
""")
@TestOutline(Scenario)
@Examples('func expected_result exitcode int_type min max', Examples_list)
def math_int_table(self, func, expected_result, exitcode, int_type, min, max, node=None):
"""Check mathematical functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = f'Nullable({int_type})')
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
for value in [1, max, min]:
with When(f"I insert the output of {func} with {int_type} using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT {func} to{int_type}(\'{value}\'))",
exitcode=exitcode, message = 'Exception:')
else:
for value in [1, max, min]:
with And(f"I insert the output of {func} with {int_type} using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT to{int_type}OrZero( toString({func} to{int_type}(\'{value}\'))))")
with Then(f"I check the outputs of {func} with {int_type}"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Scenario)
@Examples('func expected_result exitcode', Examples_dec_list)
def math_dec_inline(self, func, expected_result, exitcode, node=None):
"""Check mathematical functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
with When(f"I check {func} with Decimal256 using 1, max, and min"):
node.query(f"SELECT {func} toDecimal256(1,0)), {func} toDecimal256(\'{max}\',0)), {func} toDecimal256(\'{min}\',0))",
exitcode=43, message = 'Exception: Illegal type ')
else:
with When(f"I check {func} with Decimal256 using 1"):
output = node.query(f"SELECT {func} toDecimal256(1,0))").output
if output == 'inf':
pass
else:
assert round(float(output)) == expected_result, error()
with And(f"I check {func} with Decimal256 using max and min"):
execute_query(f"""
SELECT {func} toDecimal256(\'{max}\',0)), {func} toDecimal256(\'{min}\',0))
""")
@TestOutline(Scenario)
@Examples('func expected_result exitcode', Examples_dec_list)
def math_dec_table(self, func, expected_result, exitcode, node=None):
"""Check mathematical functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
table_name = f'table_{getuid()}'
with Given(f"I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if func in ['intExp2(', 'intExp10(', 'pow(1,', 'power(1,', 'atan2(1,', 'hypot(1,']:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT {func} toDecimal256(\'{value}\',0))",
exitcode=43, message = 'Exception: Illegal type ')
else:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 using {value} into a table"):
node.query(f"INSERT INTO {table_name} SELECT toDecimal256OrZero( toString({func} toDecimal256(\'{value}\',0))),0)")
with Then(f"I check the outputs of {func} with Decimal256"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("mathematical")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Mathematical_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Mathematical_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that mathematical functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=math_int_inline)
Scenario(run=math_int_table)
Scenario(run=math_dec_inline)
Scenario(run=math_dec_table)

View File

@ -0,0 +1,119 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('isNull(', 0),
('isNotNull(', 1),
('coalesce(', 1),
('assumeNotNull(', 1),
('toNullable(', 1),
('ifNull(1,', 1),
('nullIf(1,', '\\N'),
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
@TestOutline(Scenario)
@Examples('func expected_result int_type min max', Examples_list)
def null_int_inline(self, func, expected_result, int_type, min, max, node=None):
"""Check null function with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
with When(f"I check {func} with {int_type}"):
output = node.query(f"SELECT {func} to{int_type}(1))").output
assert output == str(expected_result), error()
with And(f"I check {func} with {int_type} using min and max"):
execute_query(f"""
SELECT {func} to{int_type}(\'{min}\')), {func} to{int_type}(\'{max}\'))
""")
@TestOutline(Scenario)
@Examples('func expected_result int_type min max', Examples_list)
def null_int_table(self, func, expected_result, int_type, min, max, node=None):
"""Check null function with Int128, UInt128, Int256, and UInt256 using table tests.
"""
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = f'Nullable({int_type})')
for value in [1, min, max]:
with When(f"I insert the output of {func} with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func} to{int_type}(\'{value}\'))")
with Then(f"I check {func} with {int_type} on the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestOutline(Scenario)
@Examples('func expected_result', funcs)
def null_dec_inline(self, func, expected_result, node=None):
"""Check null function with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
with When(f"I check {func} with Decimal256"):
output = node.query(f"SELECT {func} toDecimal256(1,0))").output
assert output == str(expected_result), error()
with And(f"I check {func} with Decimal256 using min and max"):
execute_query(f"""
SELECT {func} toDecimal256(\'{min}\',0)), {func} toDecimal256(\'{max}\',0))
""")
@TestOutline(Scenario)
@Examples('func expected_result', funcs)
def null_dec_table(self, func, expected_result, node=None):
"""Check null function with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = 'Nullable(Decimal256(0))')
for value in [1, min, max]:
with When(f"I insert the output of {func} with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT {func} toDecimal256(\'{value}\',0))")
with Then(f"I check {func} with Decimal256 on the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
@TestFeature
@Name("null")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Null("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that null functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=null_int_inline)
Scenario(run=null_int_table)
Scenario(run=null_dec_inline)
Scenario(run=null_dec_table)

View File

@ -0,0 +1,191 @@
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
funcs = [
('ceil', 1, True),
('floor', 1, True),
('trunc', 1, True),
('round', 1, True),
('roundBankers', 1, True),
('roundToExp2', 1, False),
('roundDuration', 1, True),
('roundAge', 17, True),
('roundDown', 1, False)
]
Examples_list = [tuple(list(func)+list(data_type)+[Name(f'{func[0]} - {data_type[0]}')]) for func in funcs for data_type in data_types]
Examples_dec_list = [tuple(list(func)+[Name(f'{func[0]} - Decimal256')]) for func in funcs]
@TestOutline(Scenario)
@Examples('func expected_result supported int_type min max', Examples_list)
def round_int_inline(self, func, expected_result, supported, int_type, min, max, node=None):
"""Check rounding functions with Int128, UInt128, Int256, and UInt256 using inline tests.
"""
if node is None:
node = self.context.node
if func is 'roundDown':
with When(f"I check roundDown with {int_type}"):
node.query(f"SELECT roundDown(to{int_type}(1), [0,2]), roundDown(to{int_type}(\'{max}\'), [0,2]), roundDown(to{int_type}(\'{min}\'), [0,2])",
exitcode=44, message=f'Exception: Illegal column {int_type} of first argument of function roundDown')
elif supported:
with When(f"I check {func} with {int_type}"):
output = node.query(f"SELECT {func}(to{int_type}(1))").output
assert output == str(expected_result), error()
with And(f'I check {func} with {int_type} using min and max values'):
execute_query(f"""
SELECT {func}(to{int_type}(\'{min}\')), {func}(to{int_type}(\'{max}\'))
""")
else:
with When(f"I check {func} with {int_type}"):
node.query(f"SELECT {func}(to{int_type}(1)), {func}(to{int_type}(\'{max}\')), {func}(to{int_type}(\'{min}\'))",
exitcode=48, message=f'Exception: {func}() for big integers is not implemented:')
@TestOutline(Scenario)
@Examples('func expected_result supported int_type min max', Examples_list)
def round_int_table(self, func, expected_result, supported, int_type, min, max, node=None):
"""Check rounding functions with Int128, UInt128, Int256, and UInt256 using table tests.
"""
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = int_type)
if func is 'roundDown':
for value in [1,max,min]:
with When(f"I check roundDown with {int_type} and {value}"):
node.query(f"INSERT INTO {table_name} SELECT roundDown(to{int_type}(\'{value}\'), [0,2])",
exitcode=44, message=f'Exception: Illegal column {int_type} of first argument of function roundDown')
elif supported:
for value in [1,max,min]:
with When(f"I insert the output of {func} with {int_type} and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(\'{value}\'))")
with Then(f"I select the output of {func} with {int_type} from the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
for value in [1,max,min]:
with When(f"I insert the output of {func} with {int_type} and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(to{int_type}(1))",
exitcode=48, message=f'Exception: {func}() for big integers is not implemented:')
@TestOutline(Scenario)
@Examples('func expected_result supported', Examples_dec_list)
def round_dec_inline(self, func, expected_result, supported, node=None):
"""Check rounding functions with Decimal256 using inline tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
if node is None:
node = self.context.node
if func is 'roundDown':
with When(f"I check roundDown with Decimal256"):
node.query(f"""SELECT roundDown(toDecimal256(1,0), [toDecimal256(0,0),toDecimal256(2,0)]),
roundDown(toDecimal256(\'{max}\',0), [toDecimal256(0,0),toDecimal256(2,0)]),
roundDown(toDecimal256(\'{min}\',0), [toDecimal256(0,0),toDecimal256(2,0)])""",
exitcode=44, message=f'Exception: Illegal column Decimal256 of first argument of function roundDown')
elif func not in ['roundDuration', 'roundAge', 'roundToExp2']:
with When(f"I check {func} with Decimal256"):
output = node.query(f"SELECT {func}(toDecimal256(1,0))").output
assert output == str(expected_result), error()
with And(f'I check {func} with Decimal256 using min and max values'):
execute_query(f"""
SELECT {func}(toDecimal256(\'{min}\',0)), {func}(toDecimal256(\'{max}\',0))
""")
else:
with When(f"I check {func} with Decimal256"):
node.query(f"SELECT {func}(toDecimal256(1,0)), {func}(toDecimal256(\'{max}\',0)), {func}(toDecimal256(\'{min}\',0))",
exitcode=43, message=f'Exception: Illegal type Decimal(76, 0)')
@TestOutline(Scenario)
@Examples('func expected_result supported', Examples_dec_list)
def round_dec_table(self, func, expected_result, supported, node=None):
"""Check rounding functions with Decimal256 using table tests.
"""
min = Decimal256_min_max[0]
max = Decimal256_min_max[1]
table_name = f"table_{getuid()}"
if node is None:
node = self.context.node
with Given("I have a table"):
table(name = table_name, data_type = 'Decimal256(0)')
if func is 'roundDown':
for value in [1, max, min]:
with When(f"I check roundDown with Decimal256 and {value}"):
node.query(f"INSERT INTO {table_name} SELECT roundDown(toDecimal256(\'{value}\',0), [toDecimal256(0,0),toDecimal256(2,0)])",
exitcode=44, message=f'Exception: Illegal column Decimal256 of first argument of function roundDown')
elif func not in ['roundDuration', 'roundAge', 'roundToExp2']:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0))")
with Then(f"I select the output of {func} with Decimal256 from the table"):
execute_query(f"""
SELECT * FROM {table_name} ORDER BY a ASC
""")
else:
for value in [1, max, min]:
with When(f"I insert the output of {func} with Decimal256 and {value} into the table"):
node.query(f"INSERT INTO {table_name} SELECT {func}(toDecimal256(\'{value}\',0))",
exitcode=43, message=f'Exception: Illegal type Decimal(76, 0)')
@TestFeature
@Name("rounding")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Int_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Int_NotSupported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Dec_Supported("1.0"),
RQ_SRS_020_ClickHouse_Extended_Precision_Rounding_Dec_NotSupported("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that rounding functions work with extended precision data types.
"""
self.context.node = self.context.cluster.node(node)
self.context.mysql_node = self.context.cluster.node(mysql_node)
with allow_experimental_bigint(self.context.node):
Scenario(run=round_int_inline)
Scenario(run=round_int_table)
Scenario(run=round_dec_inline)
Scenario(run=round_dec_table)

View File

@ -0,0 +1,35 @@
from testflows.core import *
from testflows.asserts import error
from contextlib import contextmanager
from extended_precision_data_types.requirements import *
from extended_precision_data_types.common import *
@TestFeature
@Name("table")
@Requirements(
RQ_SRS_020_ClickHouse_Extended_Precision_Create_Table("1.0"),
)
def feature(self, node="clickhouse1", mysql_node="mysql1", stress=None, parallel=None):
"""Check that clickhouse is able to create a table with extended precision data types.
"""
node = self.context.cluster.node(node)
table_name = f"table_{getuid()}"
with allow_experimental_bigint(node):
try:
with When("I create a table with Int128, UInt128, Int256, UInt256, Decimal256"):
node.query(f"CREATE TABLE {table_name}(a Int128, b UInt128, c Int256, d UInt256, e Decimal256(0)) ENGINE = Memory")
with And("I insert values into the table"):
node.query(f"INSERT INTO {table_name} VALUES (toInt128(1), toUInt128(1), toInt256(1), toUInt256(1), toDecimal256(1,0))")
with Then("I select from the table"):
output = node.query(f"SELECT * FROM {table_name}").output
assert output == '1\t1\t1\t1\t1', error()
finally:
with Finally("I drop the table"):
node.query(f"DROP TABLE IF EXISTS {table_name}")

View File

@ -30,6 +30,7 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
run_scenario(pool, tasks, Feature(test=load("window_functions.regression", "regression")), args)
run_scenario(pool, tasks, Feature(test=load("datetime64_extended_range.regression", "regression")), args)
#run_scenario(pool, tasks, Feature(test=load("kerberos.regression", "regression")), args)
run_scenario(pool, tasks, Feature(test=load("extended_precision_data_types.regression", "regression")), args)
finally:
join(tasks)