mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'master' into i24461
This commit is contained in:
commit
d813d03a6a
@ -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.
|
||||
|
@ -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/
|
||||
|
@ -199,7 +199,7 @@ Provides possibility to start background move data according to [TTL table expre
|
||||
Returns `Ok.` even if table doesn’t exist. Returns error when database doesn’t exist:
|
||||
|
||||
``` sql
|
||||
SYSTEM STOP MOVES [[db.]merge_tree_family_table_name]
|
||||
SYSTEM START MOVES [[db.]merge_tree_family_table_name]
|
||||
```
|
||||
|
||||
## Managing ReplicatedMergeTree Tables {#query-language-system-replicated}
|
||||
|
@ -101,6 +101,7 @@ endif()
|
||||
|
||||
if (USE_HDFS)
|
||||
add_headers_and_sources(dbms Storages/HDFS)
|
||||
add_headers_and_sources(dbms Disks/HDFS)
|
||||
endif()
|
||||
|
||||
list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD})
|
||||
|
@ -11,7 +11,8 @@ struct DiskType
|
||||
{
|
||||
Local,
|
||||
RAM,
|
||||
S3
|
||||
S3,
|
||||
HDFS
|
||||
};
|
||||
static String toString(Type disk_type)
|
||||
{
|
||||
@ -23,10 +24,11 @@ struct DiskType
|
||||
return "memory";
|
||||
case Type::S3:
|
||||
return "s3";
|
||||
case Type::HDFS:
|
||||
return "hdfs";
|
||||
}
|
||||
__builtin_unreachable();
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
194
src/Disks/HDFS/DiskHDFS.cpp
Normal file
194
src/Disks/HDFS/DiskHDFS.cpp
Normal file
@ -0,0 +1,194 @@
|
||||
#include <Disks/HDFS/DiskHDFS.h>
|
||||
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Disks/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
class HDFSPathKeeper : public RemoteFSPathKeeper
|
||||
{
|
||||
public:
|
||||
using Chunk = std::vector<std::string>;
|
||||
using Chunks = std::list<Chunk>;
|
||||
|
||||
explicit HDFSPathKeeper(size_t chunk_limit_) : RemoteFSPathKeeper(chunk_limit_) {}
|
||||
|
||||
void addPath(const String & path) override
|
||||
{
|
||||
if (chunks.empty() || chunks.back().size() >= chunk_limit)
|
||||
{
|
||||
chunks.push_back(Chunks::value_type());
|
||||
chunks.back().reserve(chunk_limit);
|
||||
}
|
||||
chunks.back().push_back(path.data());
|
||||
}
|
||||
|
||||
void removePaths(const std::function<void(Chunk &&)> & remove_chunk_func)
|
||||
{
|
||||
for (auto & chunk : chunks)
|
||||
remove_chunk_func(std::move(chunk));
|
||||
}
|
||||
|
||||
private:
|
||||
Chunks chunks;
|
||||
};
|
||||
|
||||
|
||||
/// Reads data from HDFS using stored paths in metadata.
|
||||
class ReadIndirectBufferFromHDFS final : public ReadIndirectBufferFromRemoteFS<ReadBufferFromHDFS>
|
||||
{
|
||||
public:
|
||||
ReadIndirectBufferFromHDFS(
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
const String & hdfs_uri_,
|
||||
DiskHDFS::Metadata metadata_,
|
||||
size_t buf_size_)
|
||||
: ReadIndirectBufferFromRemoteFS<ReadBufferFromHDFS>(metadata_)
|
||||
, config(config_)
|
||||
, buf_size(buf_size_)
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2);
|
||||
hdfs_directory = hdfs_uri_.substr(begin_of_path);
|
||||
hdfs_uri = hdfs_uri_.substr(0, begin_of_path);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromHDFS> createReadBuffer(const String & path) override
|
||||
{
|
||||
return std::make_unique<ReadBufferFromHDFS>(hdfs_uri, hdfs_directory + path, config, buf_size);
|
||||
}
|
||||
|
||||
private:
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
String hdfs_uri;
|
||||
String hdfs_directory;
|
||||
size_t buf_size;
|
||||
};
|
||||
|
||||
|
||||
DiskHDFS::DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
const String & metadata_path_,
|
||||
const Poco::Util::AbstractConfiguration & config_)
|
||||
: IDiskRemote(disk_name_, hdfs_root_path_, metadata_path_, "DiskHDFS", settings_->thread_pool_size)
|
||||
, config(config_)
|
||||
, hdfs_builder(createHDFSBuilder(hdfs_root_path_, config))
|
||||
, hdfs_fs(createHDFSFS(hdfs_builder.get()))
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskHDFS::readFile(const String & path, size_t buf_size, size_t, size_t, size_t, MMappedFileCache *) const
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
|
||||
LOG_DEBUG(log,
|
||||
"Read from file by path: {}. Existing HDFS objects: {}",
|
||||
backQuote(metadata_path + path), metadata.remote_fs_objects.size());
|
||||
|
||||
auto reader = std::make_unique<ReadIndirectBufferFromHDFS>(config, remote_fs_root_path, metadata, buf_size);
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(reader), settings->min_bytes_for_seek);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode)
|
||||
{
|
||||
auto metadata = readOrCreateMetaForWriting(path, mode);
|
||||
|
||||
/// Path to store new HDFS object.
|
||||
auto file_name = getRandomName();
|
||||
auto hdfs_path = remote_fs_root_path + file_name;
|
||||
|
||||
LOG_DEBUG(log, "{} to file by path: {}. HDFS path: {}", mode == WriteMode::Rewrite ? "Write" : "Append",
|
||||
backQuote(metadata_path + path), remote_fs_root_path + hdfs_path);
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(hdfs_path,
|
||||
config, buf_size,
|
||||
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS<WriteBufferFromHDFS>>(std::move(hdfs_buffer),
|
||||
std::move(metadata),
|
||||
file_name);
|
||||
}
|
||||
|
||||
|
||||
RemoteFSPathKeeperPtr DiskHDFS::createFSPathKeeper() const
|
||||
{
|
||||
return std::make_shared<HDFSPathKeeper>(settings->objects_chunk_size_to_delete);
|
||||
}
|
||||
|
||||
|
||||
void DiskHDFS::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper)
|
||||
{
|
||||
auto * hdfs_paths_keeper = dynamic_cast<HDFSPathKeeper *>(fs_paths_keeper.get());
|
||||
if (hdfs_paths_keeper)
|
||||
hdfs_paths_keeper->removePaths([&](std::vector<std::string> && chunk)
|
||||
{
|
||||
for (const auto & hdfs_object_path : chunk)
|
||||
{
|
||||
const String & hdfs_path = hdfs_object_path;
|
||||
const size_t begin_of_path = hdfs_path.find('/', hdfs_path.find("//") + 2);
|
||||
|
||||
/// Add path from root to file name
|
||||
int res = hdfsDelete(hdfs_fs.get(), hdfs_path.substr(begin_of_path).c_str(), 0);
|
||||
if (res == -1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "HDFSDelete failed with path: " + hdfs_path);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
std::unique_ptr<DiskHDFSSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
return std::make_unique<DiskHDFSSettings>(
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000));
|
||||
}
|
||||
}
|
||||
|
||||
void registerDiskHDFS(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextConstPtr context_) -> DiskPtr
|
||||
{
|
||||
Poco::File disk{context_->getPath() + "disks/" + name};
|
||||
disk.createDirectories();
|
||||
|
||||
String uri{config.getString(config_prefix + ".endpoint")};
|
||||
|
||||
if (uri.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
|
||||
|
||||
String metadata_path = context_->getPath() + "disks/" + name + "/";
|
||||
|
||||
return std::make_shared<DiskHDFS>(
|
||||
name, uri,
|
||||
getSettings(config, config_prefix),
|
||||
metadata_path, config);
|
||||
};
|
||||
|
||||
factory.registerDiskType("hdfs", creator);
|
||||
}
|
||||
|
||||
}
|
72
src/Disks/HDFS/DiskHDFS.h
Normal file
72
src/Disks/HDFS/DiskHDFS.h
Normal file
@ -0,0 +1,72 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct DiskHDFSSettings
|
||||
{
|
||||
size_t min_bytes_for_seek;
|
||||
int thread_pool_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
|
||||
DiskHDFSSettings(
|
||||
int min_bytes_for_seek_,
|
||||
int thread_pool_size_,
|
||||
int objects_chunk_size_to_delete_)
|
||||
: min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, thread_pool_size(thread_pool_size_)
|
||||
, objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {}
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in HDFS and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains HDFS object key with actual data.
|
||||
*/
|
||||
class DiskHDFS final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using SettingsPtr = std::unique_ptr<DiskHDFSSettings>;
|
||||
|
||||
DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
const String & metadata_path_,
|
||||
const Poco::Util::AbstractConfiguration & config_);
|
||||
|
||||
DiskType::Type getType() const override { return DiskType::Type::HDFS; }
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
size_t estimated_size,
|
||||
size_t aio_threshold,
|
||||
size_t mmap_threshold,
|
||||
MMappedFileCache * mmap_cache) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String & path, size_t buf_size, WriteMode mode) override;
|
||||
|
||||
void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) override;
|
||||
|
||||
RemoteFSPathKeeperPtr createFSPathKeeper() const override;
|
||||
|
||||
private:
|
||||
String getRandomName() { return toString(UUIDHelpers::generateV4()); }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
|
||||
HDFSBuilderWrapper hdfs_builder;
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
};
|
||||
|
||||
}
|
@ -7,6 +7,7 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Disks/Executor.h>
|
||||
#include <Disks/DiskType.h>
|
||||
#include "Disks/Executor.h"
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
@ -178,17 +179,17 @@ public:
|
||||
virtual void removeRecursive(const String & path) = 0;
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or if directory is not empty.
|
||||
/// Differs from removeFile for S3 disks
|
||||
/// Differs from removeFile for S3/HDFS disks
|
||||
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
|
||||
virtual void removeSharedFile(const String & path, bool) { removeFile(path); }
|
||||
|
||||
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
|
||||
/// Differs from removeRecursive for S3 disks
|
||||
/// Differs from removeRecursive for S3/HDFS disks
|
||||
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
|
||||
virtual void removeSharedRecursive(const String & path, bool) { removeRecursive(path); }
|
||||
|
||||
/// Remove file or directory if it exists.
|
||||
/// Differs from removeFileIfExists for S3 disks
|
||||
/// Differs from removeFileIfExists for S3/HDFS disks
|
||||
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
|
||||
virtual void removeSharedFileIfExists(const String & path, bool) { removeFileIfExists(path); }
|
||||
|
||||
|
487
src/Disks/IDiskRemote.cpp
Normal file
487
src/Disks/IDiskRemote.cpp
Normal file
@ -0,0 +1,487 @@
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/createHardLink.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
extern const int PATH_ACCESS_DENIED;;
|
||||
extern const int CANNOT_DELETE_DIRECTORY;
|
||||
}
|
||||
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
IDiskRemote::Metadata::Metadata(
|
||||
const String & remote_fs_root_path_,
|
||||
const String & disk_path_,
|
||||
const String & metadata_file_path_,
|
||||
bool create)
|
||||
: remote_fs_root_path(remote_fs_root_path_)
|
||||
, disk_path(disk_path_)
|
||||
, metadata_file_path(metadata_file_path_)
|
||||
, total_size(0), remote_fs_objects(0), ref_count(0)
|
||||
{
|
||||
if (create)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */
|
||||
|
||||
UInt32 version;
|
||||
readIntText(version, buf);
|
||||
|
||||
if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}",
|
||||
disk_path + metadata_file_path, toString(version), toString(VERSION_READ_ONLY_FLAG));
|
||||
|
||||
assertChar('\n', buf);
|
||||
|
||||
UInt32 remote_fs_objects_count;
|
||||
readIntText(remote_fs_objects_count, buf);
|
||||
assertChar('\t', buf);
|
||||
readIntText(total_size, buf);
|
||||
assertChar('\n', buf);
|
||||
remote_fs_objects.resize(remote_fs_objects_count);
|
||||
|
||||
for (size_t i = 0; i < remote_fs_objects_count; ++i)
|
||||
{
|
||||
String remote_fs_object_path;
|
||||
size_t remote_fs_object_size;
|
||||
readIntText(remote_fs_object_size, buf);
|
||||
assertChar('\t', buf);
|
||||
readEscapedString(remote_fs_object_path, buf);
|
||||
if (version == VERSION_ABSOLUTE_PATHS)
|
||||
{
|
||||
if (!boost::algorithm::starts_with(remote_fs_object_path, remote_fs_root_path))
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Path in metadata does not correspond S3 root path. Path: {}, root path: {}, disk path: {}",
|
||||
remote_fs_object_path, remote_fs_root_path, disk_path_);
|
||||
|
||||
remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size());
|
||||
}
|
||||
assertChar('\n', buf);
|
||||
remote_fs_objects[i] = {remote_fs_object_path, remote_fs_object_size};
|
||||
}
|
||||
|
||||
readIntText(ref_count, buf);
|
||||
assertChar('\n', buf);
|
||||
|
||||
if (version >= VERSION_READ_ONLY_FLAG)
|
||||
{
|
||||
readBoolText(read_only, buf);
|
||||
assertChar('\n', buf);
|
||||
}
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
throw;
|
||||
|
||||
throw Exception("Failed to read metadata file", e, ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
}
|
||||
|
||||
void IDiskRemote::Metadata::addObject(const String & path, size_t size)
|
||||
{
|
||||
total_size += size;
|
||||
remote_fs_objects.emplace_back(path, size);
|
||||
}
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void IDiskRemote::Metadata::save(bool sync)
|
||||
{
|
||||
WriteBufferFromFile buf(disk_path + metadata_file_path, 1024);
|
||||
|
||||
writeIntText(VERSION_RELATIVE_PATHS, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeIntText(remote_fs_objects.size(), buf);
|
||||
writeChar('\t', buf);
|
||||
writeIntText(total_size, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : remote_fs_objects)
|
||||
{
|
||||
writeIntText(remote_fs_object_size, buf);
|
||||
writeChar('\t', buf);
|
||||
writeEscapedString(remote_fs_object_path, buf);
|
||||
writeChar('\n', buf);
|
||||
}
|
||||
|
||||
writeIntText(ref_count, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeBoolText(read_only, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
buf.finalize();
|
||||
if (sync)
|
||||
buf.sync();
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readOrCreateMetaForWriting(const String & path, WriteMode mode)
|
||||
{
|
||||
bool exist = exists(path);
|
||||
if (exist)
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
if (metadata.read_only)
|
||||
throw Exception("File is read-only: " + path, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
|
||||
if (mode == WriteMode::Rewrite)
|
||||
removeFile(path); /// Remove for re-write.
|
||||
else
|
||||
return metadata;
|
||||
}
|
||||
|
||||
auto metadata = createMeta(path);
|
||||
/// Save empty metadata to disk to have ability to get file size while buffer is not finalized.
|
||||
metadata.save();
|
||||
|
||||
return metadata;
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readMeta(const String & path) const
|
||||
{
|
||||
return Metadata(remote_fs_root_path, metadata_path, path);
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::createMeta(const String & path) const
|
||||
{
|
||||
return Metadata(remote_fs_root_path, metadata_path, path, true);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper)
|
||||
{
|
||||
LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_path + path));
|
||||
|
||||
Poco::File file(metadata_path + path);
|
||||
|
||||
if (!file.isFile())
|
||||
throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path);
|
||||
|
||||
try
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
|
||||
/// If there is no references - delete content from remote FS.
|
||||
if (metadata.ref_count == 0)
|
||||
{
|
||||
file.remove();
|
||||
for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects)
|
||||
fs_paths_keeper->addPath(remote_fs_root_path + remote_fs_object_path);
|
||||
}
|
||||
else /// In other case decrement number of references, save metadata and delete file.
|
||||
{
|
||||
--metadata.ref_count;
|
||||
metadata.save();
|
||||
file.remove();
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// If it's impossible to read meta - just remove it from FS.
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
{
|
||||
LOG_WARNING(log,
|
||||
"Metadata file {} can't be read by reason: {}. Removing it forcibly.",
|
||||
backQuote(path), e.nested() ? e.nested()->message() : e.message());
|
||||
file.remove();
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
Poco::File file(metadata_path + path);
|
||||
if (file.isFile())
|
||||
{
|
||||
removeMeta(path, fs_paths_keeper);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it{iterateDirectory(path)}; it->isValid(); it->next())
|
||||
removeMetaRecursive(it->path(), fs_paths_keeper);
|
||||
file.remove();
|
||||
}
|
||||
}
|
||||
|
||||
DiskPtr DiskRemoteReservation::getDisk(size_t i) const
|
||||
{
|
||||
if (i != 0)
|
||||
throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX);
|
||||
return disk;
|
||||
}
|
||||
|
||||
|
||||
void DiskRemoteReservation::update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
disk->reserved_bytes -= size;
|
||||
size = new_size;
|
||||
disk->reserved_bytes += size;
|
||||
}
|
||||
|
||||
|
||||
DiskRemoteReservation::~DiskRemoteReservation()
|
||||
{
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(disk->log, "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->reserved_bytes -= size;
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(disk->log, "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::IDiskRemote(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & metadata_path_,
|
||||
const String & log_name_,
|
||||
size_t thread_pool_size)
|
||||
: IDisk(std::make_unique<AsyncExecutor>(log_name_, thread_pool_size))
|
||||
, log(&Poco::Logger::get(log_name_))
|
||||
, name(name_)
|
||||
, remote_fs_root_path(remote_fs_root_path_)
|
||||
, metadata_path(metadata_path_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::exists(const String & path) const
|
||||
{
|
||||
return Poco::File(metadata_path + path).exists();
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::isFile(const String & path) const
|
||||
{
|
||||
return Poco::File(metadata_path + path).isFile();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createFile(const String & path)
|
||||
{
|
||||
/// Create empty metadata file.
|
||||
auto metadata = createMeta(path);
|
||||
metadata.save();
|
||||
}
|
||||
|
||||
|
||||
size_t IDiskRemote::getFileSize(const String & path) const
|
||||
{
|
||||
auto metadata = readMeta(path);
|
||||
return metadata.total_size;
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::moveFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
|
||||
Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::replaceFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
{
|
||||
const String tmp_path = to_path + ".old";
|
||||
moveFile(to_path, tmp_path);
|
||||
moveFile(from_path, to_path);
|
||||
removeFile(tmp_path);
|
||||
}
|
||||
else
|
||||
moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeFileIfExists(const String & path)
|
||||
{
|
||||
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
|
||||
if (Poco::File(metadata_path + path).exists())
|
||||
{
|
||||
removeMeta(path, fs_paths_keeper);
|
||||
removeFromRemoteFS(fs_paths_keeper);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeSharedFile(const String & path, bool keep_in_remote_fs)
|
||||
{
|
||||
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
|
||||
removeMeta(path, fs_paths_keeper);
|
||||
if (!keep_in_remote_fs)
|
||||
removeFromRemoteFS(fs_paths_keeper);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeSharedRecursive(const String & path, bool keep_in_remote_fs)
|
||||
{
|
||||
RemoteFSPathKeeperPtr fs_paths_keeper = createFSPathKeeper();
|
||||
removeMetaRecursive(path, fs_paths_keeper);
|
||||
if (!keep_in_remote_fs)
|
||||
removeFromRemoteFS(fs_paths_keeper);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::setReadOnly(const String & path)
|
||||
{
|
||||
/// We should store read only flag inside metadata file (instead of using FS flag),
|
||||
/// because we modify metadata file when create hard-links from it.
|
||||
auto metadata = readMeta(path);
|
||||
metadata.read_only = true;
|
||||
metadata.save();
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::isDirectory(const String & path) const
|
||||
{
|
||||
return Poco::File(metadata_path + path).isDirectory();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createDirectory(const String & path)
|
||||
{
|
||||
Poco::File(metadata_path + path).createDirectory();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createDirectories(const String & path)
|
||||
{
|
||||
Poco::File(metadata_path + path).createDirectories();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::clearDirectory(const String & path)
|
||||
{
|
||||
for (auto it{iterateDirectory(path)}; it->isValid(); it->next())
|
||||
if (isFile(it->path()))
|
||||
removeFile(it->path());
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeDirectory(const String & path)
|
||||
{
|
||||
Poco::File(metadata_path + path).remove();
|
||||
}
|
||||
|
||||
|
||||
DiskDirectoryIteratorPtr IDiskRemote::iterateDirectory(const String & path)
|
||||
{
|
||||
return std::make_unique<RemoteDiskDirectoryIterator>(metadata_path + path, path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::listFiles(const String & path, std::vector<String> & file_names)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
file_names.push_back(it->name());
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp)
|
||||
{
|
||||
Poco::File(metadata_path + path).setLastModified(timestamp);
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp IDiskRemote::getLastModified(const String & path)
|
||||
{
|
||||
return Poco::File(metadata_path + path).getLastModified();
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createHardLink(const String & src_path, const String & dst_path)
|
||||
{
|
||||
/// Increment number of references.
|
||||
auto src = readMeta(src_path);
|
||||
++src.ref_count;
|
||||
src.save();
|
||||
|
||||
/// Create FS hardlink to metadata file.
|
||||
DB::createHardLink(metadata_path + src_path, metadata_path + dst_path);
|
||||
}
|
||||
|
||||
|
||||
ReservationPtr IDiskRemote::reserve(UInt64 bytes)
|
||||
{
|
||||
if (!tryReserve(bytes))
|
||||
return {};
|
||||
|
||||
return std::make_unique<DiskRemoteReservation>(std::static_pointer_cast<IDiskRemote>(shared_from_this()), bytes);
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::tryReserve(UInt64 bytes)
|
||||
{
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_DEBUG(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return true;
|
||||
}
|
||||
|
||||
auto available_space = getAvailableSpace();
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_DEBUG(log, "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
283
src/Disks/IDiskRemote.h
Normal file
283
src/Disks/IDiskRemote.h
Normal file
@ -0,0 +1,283 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <atomic>
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include "Disks/Executor.h"
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <utility>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Helper class to collect paths into chunks of maximum size.
|
||||
/// For s3 it is Aws::vector<ObjectIdentifier>, for hdfs it is std::vector<std::string>.
|
||||
class RemoteFSPathKeeper
|
||||
{
|
||||
public:
|
||||
RemoteFSPathKeeper(size_t chunk_limit_) : chunk_limit(chunk_limit_) {}
|
||||
|
||||
virtual ~RemoteFSPathKeeper() = default;
|
||||
|
||||
virtual void addPath(const String & path) = 0;
|
||||
|
||||
protected:
|
||||
size_t chunk_limit;
|
||||
};
|
||||
|
||||
using RemoteFSPathKeeperPtr = std::shared_ptr<RemoteFSPathKeeper>;
|
||||
|
||||
|
||||
/// Base Disk class for remote FS's, which are not posix-compatible (DiskS3 and DiskHDFS)
|
||||
class IDiskRemote : public IDisk
|
||||
{
|
||||
|
||||
friend class DiskRemoteReservation;
|
||||
|
||||
public:
|
||||
IDiskRemote(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & metadata_path_,
|
||||
const String & log_name_,
|
||||
size_t thread_pool_size);
|
||||
|
||||
struct Metadata;
|
||||
|
||||
const String & getName() const final override { return name; }
|
||||
|
||||
const String & getPath() const final override { return metadata_path; }
|
||||
|
||||
Metadata readMeta(const String & path) const;
|
||||
|
||||
Metadata createMeta(const String & path) const;
|
||||
|
||||
Metadata readOrCreateMetaForWriting(const String & path, WriteMode mode);
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
|
||||
void removeFileIfExists(const String & path) override;
|
||||
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false); }
|
||||
|
||||
void removeSharedFile(const String & path, bool keep_in_remote_fs) override;
|
||||
|
||||
void removeSharedRecursive(const String & path, bool keep_in_remote_fs) override;
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
void removeDirectory(const String & path) override;
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
virtual void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) = 0;
|
||||
|
||||
virtual RemoteFSPathKeeperPtr createFSPathKeeper() const = 0;
|
||||
|
||||
protected:
|
||||
Poco::Logger * log;
|
||||
const String name;
|
||||
const String remote_fs_root_path;
|
||||
|
||||
const String metadata_path;
|
||||
|
||||
private:
|
||||
void removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper);
|
||||
|
||||
void removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper);
|
||||
|
||||
bool tryReserve(UInt64 bytes);
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
};
|
||||
|
||||
using RemoteDiskPtr = std::shared_ptr<IDiskRemote>;
|
||||
|
||||
/// Remote FS (S3, HDFS) metadata file layout:
|
||||
/// Number of FS objects, Total size of all FS objects.
|
||||
/// Each FS object represents path where object located in FS and size of object.
|
||||
|
||||
struct IDiskRemote::Metadata
|
||||
{
|
||||
/// Metadata file version.
|
||||
static constexpr UInt32 VERSION_ABSOLUTE_PATHS = 1;
|
||||
static constexpr UInt32 VERSION_RELATIVE_PATHS = 2;
|
||||
static constexpr UInt32 VERSION_READ_ONLY_FLAG = 3;
|
||||
|
||||
using PathAndSize = std::pair<String, size_t>;
|
||||
|
||||
/// Remote FS (S3, HDFS) root path.
|
||||
const String & remote_fs_root_path;
|
||||
|
||||
/// Disk path.
|
||||
const String & disk_path;
|
||||
|
||||
/// Relative path to metadata file on local FS.
|
||||
String metadata_file_path;
|
||||
|
||||
/// Total size of all remote FS (S3, HDFS) objects.
|
||||
size_t total_size = 0;
|
||||
|
||||
/// Remote FS (S3, HDFS) objects paths and their sizes.
|
||||
std::vector<PathAndSize> remote_fs_objects;
|
||||
|
||||
/// Number of references (hardlinks) to this metadata file.
|
||||
UInt32 ref_count = 0;
|
||||
|
||||
/// Flag indicates that file is read only.
|
||||
bool read_only = false;
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
Metadata(const String & remote_fs_root_path_,
|
||||
const String & disk_path_,
|
||||
const String & metadata_file_path_,
|
||||
bool create = false);
|
||||
|
||||
void addObject(const String & path, size_t size);
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void save(bool sync = false);
|
||||
|
||||
};
|
||||
|
||||
|
||||
class RemoteDiskDirectoryIterator final : public IDiskDirectoryIterator
|
||||
{
|
||||
public:
|
||||
RemoteDiskDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {}
|
||||
|
||||
void next() override { ++iter; }
|
||||
|
||||
bool isValid() const override { return iter != Poco::DirectoryIterator(); }
|
||||
|
||||
String path() const override
|
||||
{
|
||||
if (iter->isDirectory())
|
||||
return folder_path + iter.name() + '/';
|
||||
else
|
||||
return folder_path + iter.name();
|
||||
}
|
||||
|
||||
String name() const override { return iter.name(); }
|
||||
|
||||
private:
|
||||
Poco::DirectoryIterator iter;
|
||||
String folder_path;
|
||||
};
|
||||
|
||||
|
||||
class DiskRemoteReservation final : public IReservation
|
||||
{
|
||||
public:
|
||||
DiskRemoteReservation(const RemoteDiskPtr & disk_, UInt64 size_)
|
||||
: disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
|
||||
{
|
||||
}
|
||||
|
||||
UInt64 getSize() const override { return size; }
|
||||
|
||||
DiskPtr getDisk(size_t i) const override;
|
||||
|
||||
Disks getDisks() const override { return {disk}; }
|
||||
|
||||
void update(UInt64 new_size) override;
|
||||
|
||||
~DiskRemoteReservation() override;
|
||||
|
||||
private:
|
||||
RemoteDiskPtr disk;
|
||||
UInt64 size;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
|
||||
/// Runs tasks asynchronously using thread pool.
|
||||
class AsyncExecutor : public Executor
|
||||
{
|
||||
public:
|
||||
explicit AsyncExecutor(const String & name_, int thread_pool_size)
|
||||
: name(name_)
|
||||
, pool(ThreadPool(thread_pool_size)) {}
|
||||
|
||||
std::future<void> execute(std::function<void()> task) override
|
||||
{
|
||||
auto promise = std::make_shared<std::promise<void>>();
|
||||
pool.scheduleOrThrowOnError(
|
||||
[promise, task]()
|
||||
{
|
||||
try
|
||||
{
|
||||
task();
|
||||
promise->set_value();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Failed to run async task");
|
||||
|
||||
try
|
||||
{
|
||||
promise->set_exception(std::current_exception());
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
});
|
||||
|
||||
return promise->get_future();
|
||||
}
|
||||
|
||||
void setMaxThreads(size_t threads)
|
||||
{
|
||||
pool.setMaxThreads(threads);
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
ThreadPool pool;
|
||||
};
|
||||
|
||||
}
|
128
src/Disks/ReadIndirectBufferFromRemoteFS.cpp
Normal file
128
src/Disks/ReadIndirectBufferFromRemoteFS.cpp
Normal file
@ -0,0 +1,128 @@
|
||||
#include "ReadIndirectBufferFromRemoteFS.h"
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
ReadIndirectBufferFromRemoteFS<T>::ReadIndirectBufferFromRemoteFS(
|
||||
IDiskRemote::Metadata metadata_)
|
||||
: metadata(std::move(metadata_))
|
||||
{
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
off_t ReadIndirectBufferFromRemoteFS<T>::seek(off_t offset_, int whence)
|
||||
{
|
||||
if (whence == SEEK_CUR)
|
||||
{
|
||||
/// If position within current working buffer - shift pos.
|
||||
if (!working_buffer.empty() && size_t(getPosition() + offset_) < absolute_position)
|
||||
{
|
||||
pos += offset_;
|
||||
return getPosition();
|
||||
}
|
||||
else
|
||||
{
|
||||
absolute_position += offset_;
|
||||
}
|
||||
}
|
||||
else if (whence == SEEK_SET)
|
||||
{
|
||||
/// If position within current working buffer - shift pos.
|
||||
if (!working_buffer.empty() && size_t(offset_) >= absolute_position - working_buffer.size()
|
||||
&& size_t(offset_) < absolute_position)
|
||||
{
|
||||
pos = working_buffer.end() - (absolute_position - offset_);
|
||||
return getPosition();
|
||||
}
|
||||
else
|
||||
{
|
||||
absolute_position = offset_;
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
current_buf = initialize();
|
||||
pos = working_buffer.end();
|
||||
|
||||
return absolute_position;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
std::unique_ptr<T> ReadIndirectBufferFromRemoteFS<T>::initialize()
|
||||
{
|
||||
size_t offset = absolute_position;
|
||||
for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i)
|
||||
{
|
||||
current_buf_idx = i;
|
||||
const auto & [file_path, size] = metadata.remote_fs_objects[i];
|
||||
if (size > offset)
|
||||
{
|
||||
auto buf = createReadBuffer(file_path);
|
||||
buf->seek(offset, SEEK_SET);
|
||||
return buf;
|
||||
}
|
||||
offset -= size;
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
template<typename T>
|
||||
bool ReadIndirectBufferFromRemoteFS<T>::nextImpl()
|
||||
{
|
||||
/// Find first available buffer that fits to given offset.
|
||||
if (!current_buf)
|
||||
current_buf = initialize();
|
||||
|
||||
/// If current buffer has remaining data - use it.
|
||||
if (current_buf && current_buf->next())
|
||||
{
|
||||
working_buffer = current_buf->buffer();
|
||||
absolute_position += working_buffer.size();
|
||||
return true;
|
||||
}
|
||||
|
||||
/// If there is no available buffers - nothing to read.
|
||||
if (current_buf_idx + 1 >= metadata.remote_fs_objects.size())
|
||||
return false;
|
||||
|
||||
++current_buf_idx;
|
||||
const auto & path = metadata.remote_fs_objects[current_buf_idx].first;
|
||||
|
||||
current_buf = createReadBuffer(path);
|
||||
current_buf->next();
|
||||
|
||||
working_buffer = current_buf->buffer();
|
||||
absolute_position += working_buffer.size();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
#if USE_AWS_S3
|
||||
template
|
||||
class ReadIndirectBufferFromRemoteFS<ReadBufferFromS3>;
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
template
|
||||
class ReadIndirectBufferFromRemoteFS<ReadBufferFromHDFS>;
|
||||
#endif
|
||||
|
||||
}
|
||||
|
||||
#endif
|
46
src/Disks/ReadIndirectBufferFromRemoteFS.h
Normal file
46
src/Disks/ReadIndirectBufferFromRemoteFS.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <utility>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Reads data from S3/HDFS using stored paths in metadata.
|
||||
template <typename T>
|
||||
class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase
|
||||
{
|
||||
public:
|
||||
ReadIndirectBufferFromRemoteFS(IDiskRemote::Metadata metadata_);
|
||||
|
||||
off_t seek(off_t offset_, int whence) override;
|
||||
|
||||
off_t getPosition() override { return absolute_position - available(); }
|
||||
|
||||
String getFileName() const override { return metadata.metadata_file_path; }
|
||||
|
||||
virtual std::unique_ptr<T> createReadBuffer(const String & path) = 0;
|
||||
|
||||
protected:
|
||||
IDiskRemote::Metadata metadata;
|
||||
|
||||
private:
|
||||
std::unique_ptr<T> initialize();
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
size_t absolute_position = 0;
|
||||
|
||||
size_t current_buf_idx = 0;
|
||||
|
||||
std::unique_ptr<T> current_buf;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
File diff suppressed because it is too large
Load Diff
@ -2,7 +2,6 @@
|
||||
|
||||
#include <atomic>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include "Disks/Executor.h"
|
||||
|
||||
@ -12,6 +11,7 @@
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <re2/re2.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -28,7 +28,8 @@ struct DiskS3Settings
|
||||
size_t min_bytes_for_seek_,
|
||||
bool send_metadata_,
|
||||
int thread_pool_size_,
|
||||
int list_object_keys_size_);
|
||||
int list_object_keys_size_,
|
||||
int objects_chunk_size_to_delete_);
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
size_t s3_max_single_read_retries;
|
||||
@ -38,25 +39,24 @@ struct DiskS3Settings
|
||||
bool send_metadata;
|
||||
int thread_pool_size;
|
||||
int list_object_keys_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in S3 and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains S3 object key with actual data.
|
||||
*/
|
||||
class DiskS3 : public IDisk
|
||||
class DiskS3 final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using ObjectMetadata = std::map<std::string, std::string>;
|
||||
using Futures = std::vector<std::future<void>>;
|
||||
|
||||
using SettingsPtr = std::unique_ptr<DiskS3Settings>;
|
||||
using GetDiskSettings = std::function<SettingsPtr(const Poco::Util::AbstractConfiguration &, const String, ContextConstPtr)>;
|
||||
|
||||
friend class DiskS3Reservation;
|
||||
|
||||
class AwsS3KeyKeeper;
|
||||
struct Metadata;
|
||||
struct RestoreInformation;
|
||||
|
||||
DiskS3(
|
||||
@ -67,44 +67,6 @@ public:
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_);
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
|
||||
const String & getPath() const override { return metadata_path; }
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
void moveFile(const String & from_path, const String & to_path, bool send_metadata);
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
@ -118,25 +80,16 @@ public:
|
||||
size_t buf_size,
|
||||
WriteMode mode) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
void removeFileIfExists(const String & path) override;
|
||||
void removeDirectory(const String & path) override;
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false); }
|
||||
void removeFromRemoteFS(RemoteFSPathKeeperPtr keeper) override;
|
||||
|
||||
void removeSharedFile(const String & path, bool keep_s3) override;
|
||||
void removeSharedRecursive(const String & path, bool keep_s3) override;
|
||||
RemoteFSPathKeeperPtr createFSPathKeeper() const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path, bool send_metadata);
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
void createHardLink(const String & src_path, const String & dst_path, bool send_metadata);
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
DiskType::Type getType() const override { return DiskType::Type::S3; }
|
||||
|
||||
void shutdown() override;
|
||||
@ -157,16 +110,6 @@ public:
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextConstPtr context) override;
|
||||
|
||||
private:
|
||||
bool tryReserve(UInt64 bytes);
|
||||
|
||||
void removeMeta(const String & path, AwsS3KeyKeeper & keys);
|
||||
void removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys);
|
||||
void removeAws(const AwsS3KeyKeeper & keys);
|
||||
|
||||
Metadata readOrCreateMetaForWriting(const String & path, WriteMode mode);
|
||||
Metadata readMeta(const String & path) const;
|
||||
Metadata createMeta(const String & path) const;
|
||||
|
||||
void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata);
|
||||
/// Converts revision to binary string with leading zeroes (64 bit).
|
||||
static String revisionToString(UInt64 revision);
|
||||
@ -200,19 +143,12 @@ private:
|
||||
/// Forms detached path '../../detached/part_name/' from '../../part_name/'
|
||||
static String pathToDetached(const String & source_path);
|
||||
|
||||
const String name;
|
||||
const String bucket;
|
||||
const String s3_root_path;
|
||||
const String metadata_path;
|
||||
MultiVersion<DiskS3Settings> current_settings;
|
||||
|
||||
MultiVersion<DiskS3Settings> current_settings;
|
||||
/// Gets disk settings from context.
|
||||
GetDiskSettings settings_getter;
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
|
||||
std::atomic<UInt64> revision_counter = 0;
|
||||
static constexpr UInt64 LATEST_REVISION = std::numeric_limits<UInt64>::max();
|
||||
static constexpr UInt64 UNKNOWN_REVISION = 0;
|
||||
@ -229,8 +165,6 @@ private:
|
||||
static constexpr int RESTORABLE_SCHEMA_VERSION = 1;
|
||||
/// Directories with data.
|
||||
const std::vector<String> data_roots {"data", "store"};
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("DiskS3");
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -156,7 +156,8 @@ std::unique_ptr<DiskS3Settings> getSettings(const Poco::Util::AbstractConfigurat
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getBool(config_prefix + ".send_metadata", false),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".list_object_keys_size", 1000));
|
||||
config.getInt(config_prefix + ".list_object_keys_size", 1000),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000));
|
||||
}
|
||||
|
||||
}
|
||||
@ -225,4 +226,3 @@ void registerDiskS3(DiskFactory & factory)
|
||||
void registerDiskS3(DiskFactory &) {}
|
||||
|
||||
#endif
|
||||
|
||||
|
71
src/Disks/WriteIndirectBufferFromRemoteFS.cpp
Normal file
71
src/Disks/WriteIndirectBufferFromRemoteFS.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include "WriteIndirectBufferFromRemoteFS.h"
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS.
|
||||
template <typename T>
|
||||
WriteIndirectBufferFromRemoteFS<T>::WriteIndirectBufferFromRemoteFS(
|
||||
std::unique_ptr<T> impl_,
|
||||
IDiskRemote::Metadata metadata_,
|
||||
const String & remote_fs_path_)
|
||||
: WriteBufferFromFileDecorator(std::move(impl_))
|
||||
, metadata(std::move(metadata_))
|
||||
, remote_fs_path(remote_fs_path_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
WriteIndirectBufferFromRemoteFS<T>::~WriteIndirectBufferFromRemoteFS()
|
||||
{
|
||||
try
|
||||
{
|
||||
WriteIndirectBufferFromRemoteFS::finalize();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void WriteIndirectBufferFromRemoteFS<T>::finalize()
|
||||
{
|
||||
if (finalized)
|
||||
return;
|
||||
|
||||
WriteBufferFromFileDecorator::finalize();
|
||||
|
||||
metadata.addObject(remote_fs_path, count());
|
||||
metadata.save();
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void WriteIndirectBufferFromRemoteFS<T>::sync()
|
||||
{
|
||||
if (finalized)
|
||||
metadata.save(true);
|
||||
}
|
||||
|
||||
|
||||
#if USE_AWS_S3
|
||||
template
|
||||
class WriteIndirectBufferFromRemoteFS<WriteBufferFromS3>;
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
template
|
||||
class WriteIndirectBufferFromRemoteFS<WriteBufferFromHDFS>;
|
||||
#endif
|
||||
|
||||
}
|
||||
|
||||
#endif
|
39
src/Disks/WriteIndirectBufferFromRemoteFS.h
Normal file
39
src/Disks/WriteIndirectBufferFromRemoteFS.h
Normal file
@ -0,0 +1,39 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3 || USE_HDFS
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFileDecorator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Stores data in S3/HDFS and adds the object key (S3 path) and object size to metadata file on local FS.
|
||||
template <typename T>
|
||||
class WriteIndirectBufferFromRemoteFS final : public WriteBufferFromFileDecorator
|
||||
{
|
||||
public:
|
||||
WriteIndirectBufferFromRemoteFS(
|
||||
std::unique_ptr<T> impl_,
|
||||
IDiskRemote::Metadata metadata_,
|
||||
const String & remote_fs_path_);
|
||||
|
||||
virtual ~WriteIndirectBufferFromRemoteFS() override;
|
||||
|
||||
void finalize() override;
|
||||
|
||||
void sync() override;
|
||||
|
||||
String getFileName() const override { return metadata.metadata_file_path; }
|
||||
|
||||
private:
|
||||
IDiskRemote::Metadata metadata;
|
||||
|
||||
String remote_fs_path;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -8,21 +8,33 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerDiskLocal(DiskFactory & factory);
|
||||
void registerDiskMemory(DiskFactory & factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
void registerDiskS3(DiskFactory & factory);
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
void registerDiskHDFS(DiskFactory & factory);
|
||||
#endif
|
||||
|
||||
|
||||
void registerDisks()
|
||||
{
|
||||
auto & factory = DiskFactory::instance();
|
||||
|
||||
registerDiskLocal(factory);
|
||||
registerDiskMemory(factory);
|
||||
|
||||
#if USE_AWS_S3
|
||||
registerDiskS3(factory);
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
registerDiskHDFS(factory);
|
||||
#endif
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,14 +1,15 @@
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "gtest_disk.h"
|
||||
|
||||
|
||||
#if !defined(__clang__)
|
||||
# pragma GCC diagnostic push
|
||||
# pragma GCC diagnostic ignored "-Wsuggest-override"
|
||||
#endif
|
||||
|
||||
|
||||
template <typename T>
|
||||
DB::DiskPtr createDisk();
|
||||
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/DiskMemory.h>
|
||||
#include <Disks/IDisk.h>
|
||||
|
160
src/Disks/tests/gtest_disk_hdfs.cpp
Normal file
160
src/Disks/tests/gtest_disk_hdfs.cpp
Normal file
@ -0,0 +1,160 @@
|
||||
#include <gtest/gtest.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include "gtest_disk.h"
|
||||
|
||||
|
||||
#define RUN_HDFS_TEST 0
|
||||
#if RUN_HDFS_TEST
|
||||
|
||||
#include <Disks/HDFS/DiskHDFS.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
const String hdfs_uri = "hdfs://172.20.0.2:9000/disk_test/";
|
||||
const String metadata_path = "/path/to/metadata/";
|
||||
const String config_path = "/path/to/config.xml";
|
||||
const String file_name = "test.txt";
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, RemoveFileHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
DB::HDFSBuilderWrapper builder = DB::createHDFSBuilder(hdfs_uri, *config);
|
||||
DB::HDFSFSPtr fs = DB::createHDFSFS(builder.get());
|
||||
|
||||
disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
auto metadata = disk.readMeta(file_name);
|
||||
|
||||
const String hdfs_file_name = metadata.remote_fs_objects[0].first;
|
||||
const String hdfs_file_path = "/disk_test/" + hdfs_file_name;
|
||||
|
||||
auto ret = hdfsExists(fs.get(), hdfs_file_path.data());
|
||||
EXPECT_EQ(0, ret);
|
||||
|
||||
disk.removeFile(file_name);
|
||||
ret = hdfsExists(fs.get(), hdfs_file_path.data());
|
||||
EXPECT_EQ(-1, ret);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, WriteReadHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
{
|
||||
auto out = disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
writeString("Test write to file", *out);
|
||||
}
|
||||
|
||||
{
|
||||
DB::String result;
|
||||
auto in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("Test write to file", result);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, RewriteFileHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
for (size_t i = 1; i <= 10; ++i)
|
||||
{
|
||||
std::unique_ptr<DB::WriteBuffer> out = disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
writeString("Text" + DB::toString(i), *out);
|
||||
}
|
||||
|
||||
{
|
||||
String result;
|
||||
auto in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("Text10", result);
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("", result);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, AppendFileHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
{
|
||||
std::unique_ptr<DB::WriteBuffer> out = disk.writeFile(file_name, 1024, DB::WriteMode::Append);
|
||||
writeString("Text", *out);
|
||||
for (size_t i = 0; i < 10; ++i)
|
||||
{
|
||||
writeIntText(i, *out);
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
String result, expected;
|
||||
auto in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("Text0123456789", result);
|
||||
|
||||
readString(result, *in);
|
||||
EXPECT_EQ("", result);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
|
||||
TEST(DiskTestHDFS, SeekHDFS)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration *config = new Poco::Util::XMLConfiguration(config_path);
|
||||
auto settings = std::make_unique<DB::DiskHDFSSettings>(1024 * 1024);
|
||||
auto disk = DB::DiskHDFS("disk_hdfs", hdfs_uri, std::move(settings), metadata_path, *config);
|
||||
|
||||
{
|
||||
std::unique_ptr<DB::WriteBuffer> out = disk.writeFile(file_name, 1024, DB::WriteMode::Rewrite);
|
||||
writeString("test data", *out);
|
||||
}
|
||||
|
||||
/// Test SEEK_SET
|
||||
{
|
||||
String buf(4, '0');
|
||||
std::unique_ptr<DB::SeekableReadBuffer> in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
|
||||
in->seek(5, SEEK_SET);
|
||||
|
||||
in->readStrict(buf.data(), 4);
|
||||
EXPECT_EQ("data", buf);
|
||||
}
|
||||
|
||||
/// Test SEEK_CUR
|
||||
{
|
||||
std::unique_ptr<DB::SeekableReadBuffer> in = disk.readFile(file_name, 1024, 1024, 1024, 1024, nullptr);
|
||||
String buf(4, '0');
|
||||
|
||||
in->readStrict(buf.data(), 4);
|
||||
EXPECT_EQ("test", buf);
|
||||
|
||||
// Skip whitespace
|
||||
in->seek(1, SEEK_CUR);
|
||||
|
||||
in->readStrict(buf.data(), 4);
|
||||
EXPECT_EQ("data", buf);
|
||||
}
|
||||
|
||||
disk.removeFileIfExists(file_name);
|
||||
}
|
||||
|
||||
#endif
|
@ -7,8 +7,7 @@ PEERDIR(
|
||||
)
|
||||
|
||||
|
||||
SRCS(
|
||||
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F examples | grep -v -F S3 | sed 's/^\.\// /' | sort ?>
|
||||
<? find . -name '*.cpp' | grep -v -F tests | grep -v -F examples | grep -v -F 'S3|HDFS' | sed 's/^\.\// /' | sort ?>
|
||||
)
|
||||
|
||||
END()
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <common/types.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/IFunction.h>
|
||||
@ -18,7 +19,6 @@ namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
@ -26,43 +26,6 @@ namespace ErrorCodes
|
||||
* CustomWeek Transformations.
|
||||
*/
|
||||
|
||||
static inline UInt32 dateIsNotSupported(const char * name)
|
||||
{
|
||||
throw Exception("Illegal type Date of argument for function " + std::string(name), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
/// This factor transformation will say that the function is monotone everywhere.
|
||||
struct ZeroTransform
|
||||
{
|
||||
static inline UInt16 execute(UInt16, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(UInt32, UInt8, const DateLUTImpl &) { return 0; }
|
||||
static inline UInt16 execute(Int64, UInt8, const DateLUTImpl &) { return 0; }
|
||||
};
|
||||
|
||||
struct ToWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
struct ToYearWeekImpl
|
||||
{
|
||||
static constexpr auto name = "toYearWeek";
|
||||
@ -110,10 +73,34 @@ struct ToStartOfWeekImpl
|
||||
using FactorTransform = ZeroTransform;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct Transformer
|
||||
struct ToWeekImpl
|
||||
{
|
||||
explicit Transformer(Transform transform_)
|
||||
static constexpr auto name = "toWeek";
|
||||
|
||||
static inline UInt8 execute(Int64 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
// TODO: ditch conversion to DayNum, since it doesn't support extended range.
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt32 t, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(time_zone.toDayNum(t), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
static inline UInt8 execute(UInt16 d, UInt8 week_mode, const DateLUTImpl & time_zone)
|
||||
{
|
||||
YearWeek yw = time_zone.toYearWeek(DayNum(d), week_mode);
|
||||
return yw.second;
|
||||
}
|
||||
|
||||
using FactorTransform = ToStartOfYearImpl;
|
||||
};
|
||||
|
||||
template <typename FromType, typename ToType, typename Transform>
|
||||
struct WeekTransformer
|
||||
{
|
||||
explicit WeekTransformer(Transform transform_)
|
||||
: transform(std::move(transform_))
|
||||
{}
|
||||
|
||||
@ -139,7 +126,7 @@ struct CustomWeekTransformImpl
|
||||
template <typename Transform>
|
||||
static ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/, Transform transform = {})
|
||||
{
|
||||
const auto op = Transformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
const auto op = WeekTransformer<typename FromDataType::FieldType, typename ToDataType::FieldType, Transform>{std::move(transform)};
|
||||
|
||||
UInt8 week_mode = DEFAULT_WEEK_MODE;
|
||||
if (arguments.size() > 1)
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -1,57 +1,57 @@
|
||||
#include <iostream>
|
||||
|
||||
// #include <llvm/IR/IRBuilder.h>
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
|
||||
// #include <Interpreters/JIT/CHJIT.h>
|
||||
#include <Interpreters/JIT/CHJIT.h>
|
||||
|
||||
// void test_function()
|
||||
// {
|
||||
// std::cerr << "Test function" << std::endl;
|
||||
// }
|
||||
void test_function()
|
||||
{
|
||||
std::cerr << "Test function" << std::endl;
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
(void)(argc);
|
||||
(void)(argv);
|
||||
|
||||
// auto jit = DB::CHJIT();
|
||||
auto jit = DB::CHJIT();
|
||||
|
||||
// jit.registerExternalSymbol("test_function", reinterpret_cast<void *>(&test_function));
|
||||
jit.registerExternalSymbol("test_function", reinterpret_cast<void *>(&test_function));
|
||||
|
||||
// auto compiled_module_info = jit.compileModule([](llvm::Module & module)
|
||||
// {
|
||||
// auto & context = module.getContext();
|
||||
// llvm::IRBuilder<> b (context);
|
||||
auto compiled_module_info = jit.compileModule([](llvm::Module & module)
|
||||
{
|
||||
auto & context = module.getContext();
|
||||
llvm::IRBuilder<> b (context);
|
||||
|
||||
// auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||
// auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||
auto * func_declaration_type = llvm::FunctionType::get(b.getVoidTy(), { }, /*isVarArg=*/false);
|
||||
auto * func_declaration = llvm::Function::Create(func_declaration_type, llvm::Function::ExternalLinkage, "test_function", module);
|
||||
|
||||
// auto * value_type = b.getInt64Ty();
|
||||
// auto * pointer_type = value_type->getPointerTo();
|
||||
auto * value_type = b.getInt64Ty();
|
||||
auto * pointer_type = value_type->getPointerTo();
|
||||
|
||||
// auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||
// auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||
// auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||
auto * func_type = llvm::FunctionType::get(b.getVoidTy(), { pointer_type }, /*isVarArg=*/false);
|
||||
auto * function = llvm::Function::Create(func_type, llvm::Function::ExternalLinkage, "test_name", module);
|
||||
auto * entry = llvm::BasicBlock::Create(context, "entry", function);
|
||||
|
||||
// auto * argument = function->args().begin();
|
||||
// b.SetInsertPoint(entry);
|
||||
auto * argument = function->args().begin();
|
||||
b.SetInsertPoint(entry);
|
||||
|
||||
// b.CreateCall(func_declaration);
|
||||
b.CreateCall(func_declaration);
|
||||
|
||||
// auto * load_argument = b.CreateLoad(value_type, argument);
|
||||
// auto * value = b.CreateAdd(load_argument, load_argument);
|
||||
// b.CreateRet(value);
|
||||
// });
|
||||
auto * load_argument = b.CreateLoad(value_type, argument);
|
||||
auto * value = b.CreateAdd(load_argument, load_argument);
|
||||
b.CreateRet(value);
|
||||
});
|
||||
|
||||
// for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||
// {
|
||||
// std::cerr << compiled_function_name << std::endl;
|
||||
// }
|
||||
for (const auto & compiled_function_name : compiled_module_info.compiled_functions)
|
||||
{
|
||||
std::cerr << compiled_function_name << std::endl;
|
||||
}
|
||||
|
||||
// int64_t value = 5;
|
||||
// auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction(compiled_module_info, "test_name"));
|
||||
// auto result = test_name_function(&value);
|
||||
// std::cerr << "Result " << result << std::endl;
|
||||
int64_t value = 5;
|
||||
auto * test_name_function = reinterpret_cast<int64_t (*)(int64_t *)>(jit.findCompiledFunction(compiled_module_info, "test_name"));
|
||||
auto result = test_name_function(&value);
|
||||
std::cerr << "Result " << result << std::endl;
|
||||
|
||||
return 0;
|
||||
}
|
||||
|
@ -145,6 +145,7 @@ HDFSBuilderWrapper createHDFSBuilder(const String & uri_str, const Poco::Util::A
|
||||
|
||||
hdfsBuilderSetUserName(builder.get(), user.c_str());
|
||||
}
|
||||
|
||||
hdfsBuilderSetNameNode(builder.get(), host.c_str());
|
||||
if (port != 0)
|
||||
{
|
||||
|
@ -1,13 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
#if !defined(ARCADIA_BUILD)
|
||||
#include <Common/config.h>
|
||||
#endif
|
||||
|
||||
#if USE_HDFS
|
||||
#include <memory>
|
||||
#include <type_traits>
|
||||
#include <vector>
|
||||
|
||||
#include <hdfs/hdfs.h>
|
||||
#include <hdfs/hdfs.h> // Y_IGNORE
|
||||
#include <common/types.h>
|
||||
#include <mutex>
|
||||
|
||||
|
@ -13,6 +13,8 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int NETWORK_ERROR;
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_SEEK_THROUGH_FILE;
|
||||
extern const int SEEK_POSITION_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
ReadBufferFromHDFS::~ReadBufferFromHDFS() = default;
|
||||
@ -29,6 +31,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
|
||||
HDFSBuilderWrapper builder;
|
||||
HDFSFSPtr fs;
|
||||
|
||||
off_t offset = 0;
|
||||
bool initialized = false;
|
||||
|
||||
explicit ReadBufferFromHDFSImpl(
|
||||
const std::string & hdfs_uri_,
|
||||
const std::string & hdfs_file_path_,
|
||||
@ -48,8 +53,30 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
|
||||
hdfs_uri + hdfs_file_path, std::string(hdfsGetLastError()));
|
||||
}
|
||||
|
||||
int read(char * start, size_t size) const
|
||||
~ReadBufferFromHDFSImpl()
|
||||
{
|
||||
std::lock_guard lock(hdfs_init_mutex);
|
||||
hdfsCloseFile(fs.get(), fin);
|
||||
}
|
||||
|
||||
void initialize() const
|
||||
{
|
||||
if (!offset)
|
||||
return;
|
||||
|
||||
int seek_status = hdfsSeek(fs.get(), fin, offset);
|
||||
if (seek_status != 0)
|
||||
throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Fail to seek HDFS file: {}, error: {}", hdfs_uri, std::string(hdfsGetLastError()));
|
||||
}
|
||||
|
||||
int read(char * start, size_t size)
|
||||
{
|
||||
if (!initialized)
|
||||
{
|
||||
initialize();
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
int bytes_read = hdfsRead(fs.get(), fin, start, size);
|
||||
if (bytes_read < 0)
|
||||
throw Exception(ErrorCodes::NETWORK_ERROR,
|
||||
@ -58,10 +85,25 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
|
||||
return bytes_read;
|
||||
}
|
||||
|
||||
~ReadBufferFromHDFSImpl()
|
||||
int seek(off_t offset_, int whence)
|
||||
{
|
||||
std::lock_guard lock(hdfs_init_mutex);
|
||||
hdfsCloseFile(fs.get(), fin);
|
||||
if (initialized)
|
||||
throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
if (whence != SEEK_SET)
|
||||
throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
|
||||
|
||||
if (offset_ < 0)
|
||||
throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bounds. Offset: {}", std::to_string(offset_));
|
||||
|
||||
offset = offset_;
|
||||
|
||||
return offset;
|
||||
}
|
||||
|
||||
int tell() const
|
||||
{
|
||||
return offset;
|
||||
}
|
||||
};
|
||||
|
||||
@ -73,7 +115,7 @@ ReadBufferFromHDFS::ReadBufferFromHDFS(
|
||||
const String & hdfs_file_path_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
size_t buf_size_)
|
||||
: BufferWithOwnMemory<ReadBuffer>(buf_size_)
|
||||
: BufferWithOwnMemory<SeekableReadBuffer>(buf_size_)
|
||||
, impl(std::make_unique<ReadBufferFromHDFSImpl>(hdfs_uri_, hdfs_file_path_, config_))
|
||||
{
|
||||
}
|
||||
@ -90,6 +132,18 @@ bool ReadBufferFromHDFS::nextImpl()
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
off_t ReadBufferFromHDFS::seek(off_t off, int whence)
|
||||
{
|
||||
return impl->seek(off, whence);
|
||||
}
|
||||
|
||||
|
||||
off_t ReadBufferFromHDFS::getPosition()
|
||||
{
|
||||
return impl->tell() + count();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -7,28 +7,34 @@
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <string>
|
||||
#include <memory>
|
||||
#include <hdfs/hdfs.h>
|
||||
#include <hdfs/hdfs.h> // Y_IGNORE
|
||||
#include <common/types.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Accepts HDFS path to file and opens it.
|
||||
* Closes file by himself (thus "owns" a file descriptor).
|
||||
*/
|
||||
class ReadBufferFromHDFS : public BufferWithOwnMemory<ReadBuffer>
|
||||
class ReadBufferFromHDFS : public BufferWithOwnMemory<SeekableReadBuffer>
|
||||
{
|
||||
struct ReadBufferFromHDFSImpl;
|
||||
|
||||
public:
|
||||
ReadBufferFromHDFS(const String & hdfs_uri_, const String & hdfs_file_path_,
|
||||
const Poco::Util::AbstractConfiguration &, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
const Poco::Util::AbstractConfiguration & config_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
|
||||
~ReadBufferFromHDFS() override;
|
||||
|
||||
bool nextImpl() override;
|
||||
|
||||
off_t seek(off_t offset_, int whence) override;
|
||||
|
||||
off_t getPosition() override;
|
||||
|
||||
private:
|
||||
std::unique_ptr<ReadBufferFromHDFSImpl> impl;
|
||||
};
|
||||
|
@ -64,7 +64,6 @@ public:
|
||||
struct SourcesInfo
|
||||
{
|
||||
std::vector<String> uris;
|
||||
|
||||
std::atomic<size_t> next_uri_to_read = 0;
|
||||
|
||||
bool need_path_column = false;
|
||||
|
@ -27,20 +27,24 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
|
||||
HDFSBuilderWrapper builder;
|
||||
HDFSFSPtr fs;
|
||||
|
||||
explicit WriteBufferFromHDFSImpl(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_)
|
||||
: hdfs_uri(hdfs_name_)
|
||||
, builder(createHDFSBuilder(hdfs_uri,config_))
|
||||
explicit WriteBufferFromHDFSImpl(
|
||||
const std::string & hdfs_uri_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
int flags)
|
||||
: hdfs_uri(hdfs_uri_)
|
||||
, builder(createHDFSBuilder(hdfs_uri, config_))
|
||||
, fs(createHDFSFS(builder.get()))
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const String path = hdfs_uri.substr(begin_of_path);
|
||||
|
||||
if (path.find_first_of("*?{") != std::string::npos)
|
||||
throw Exception("URI '" + hdfs_uri + "' contains globs, so the table is in readonly mode", ErrorCodes::CANNOT_OPEN_FILE);
|
||||
throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "URI '{}' contains globs, so the table is in readonly mode", hdfs_uri);
|
||||
|
||||
if (!hdfsExists(fs.get(), path.c_str()))
|
||||
throw Exception("File: " + path + " is already exists", ErrorCodes::BAD_ARGUMENTS);
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} already exists", path);
|
||||
|
||||
fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here
|
||||
fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here
|
||||
|
||||
if (fout == nullptr)
|
||||
{
|
||||
@ -76,9 +80,13 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
|
||||
}
|
||||
};
|
||||
|
||||
WriteBufferFromHDFS::WriteBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, size_t buf_size_)
|
||||
WriteBufferFromHDFS::WriteBufferFromHDFS(
|
||||
const std::string & hdfs_name_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
size_t buf_size_,
|
||||
int flags_)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buf_size_)
|
||||
, impl(std::make_unique<WriteBufferFromHDFSImpl>(hdfs_name_, config_))
|
||||
, impl(std::make_unique<WriteBufferFromHDFSImpl>(hdfs_name_, config_, flags_))
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <string>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Accepts HDFS path to file and opens it.
|
||||
@ -15,11 +16,13 @@ namespace DB
|
||||
*/
|
||||
class WriteBufferFromHDFS final : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
struct WriteBufferFromHDFSImpl;
|
||||
std::unique_ptr<WriteBufferFromHDFSImpl> impl;
|
||||
|
||||
public:
|
||||
WriteBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration &, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE);
|
||||
WriteBufferFromHDFS(
|
||||
const std::string & hdfs_name_,
|
||||
const Poco::Util::AbstractConfiguration & config_,
|
||||
size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
int flags = O_WRONLY);
|
||||
|
||||
WriteBufferFromHDFS(WriteBufferFromHDFS &&) = default;
|
||||
|
||||
@ -30,6 +33,11 @@ public:
|
||||
void sync() override;
|
||||
|
||||
void finalize() override;
|
||||
|
||||
private:
|
||||
struct WriteBufferFromHDFSImpl;
|
||||
std::unique_ptr<WriteBufferFromHDFSImpl> impl;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
||||
|
@ -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);
|
||||
|
@ -11,6 +11,10 @@
|
||||
<disk_memory>
|
||||
<type>memory</type>
|
||||
</disk_memory>
|
||||
<disk_hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>http://hdfs1:9000/data/</endpoint>
|
||||
</disk_hdfs>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
</yandex>
|
||||
|
@ -5,6 +5,7 @@ disk_types = {
|
||||
"default": "local",
|
||||
"disk_s3": "s3",
|
||||
"disk_memory": "memory",
|
||||
"disk_hdfs": "hdfs",
|
||||
}
|
||||
|
||||
|
||||
@ -12,7 +13,7 @@ disk_types = {
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True)
|
||||
cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True, with_hdfs=True)
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
@ -35,3 +36,4 @@ def test_select_by_type(cluster):
|
||||
node = cluster.instances["node"]
|
||||
for name, disk_type in list(disk_types.items()):
|
||||
assert node.query("SELECT name FROM system.disks WHERE type='" + disk_type + "'") == name + "\n"
|
||||
|
||||
|
0
tests/integration/test_log_family_hdfs/__init__.py
Normal file
0
tests/integration/test_log_family_hdfs/__init__.py
Normal file
@ -0,0 +1,12 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
</yandex>
|
@ -0,0 +1,11 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
|
||||
</hdfs>
|
||||
</disks>
|
||||
</storage_configuration>
|
||||
</yandex>
|
59
tests/integration/test_log_family_hdfs/test.py
Normal file
59
tests/integration/test_log_family_hdfs/test.py
Normal file
@ -0,0 +1,59 @@
|
||||
import logging
|
||||
import sys
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster.add_instance("node",
|
||||
main_configs=["configs/storage_conf.xml", "configs/config.d/log_conf.xml"],
|
||||
with_hdfs=True)
|
||||
logging.info("Starting cluster...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
fs.mkdirs('/clickhouse')
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def assert_objects_count(cluster, objects_count, path='data/'):
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert objects_count == len(hdfs_objects)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
"log_engine,files_overhead,files_overhead_per_insert",
|
||||
[("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)])
|
||||
def test_log_family_hdfs(cluster, log_engine, files_overhead, files_overhead_per_insert):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
node.query("CREATE TABLE hdfs_test (id UInt64) ENGINE={} SETTINGS disk = 'hdfs'".format(log_engine))
|
||||
|
||||
node.query("INSERT INTO hdfs_test SELECT number FROM numbers(5)")
|
||||
assert node.query("SELECT * FROM hdfs_test") == "0\n1\n2\n3\n4\n"
|
||||
assert_objects_count(cluster, files_overhead_per_insert + files_overhead)
|
||||
|
||||
node.query("INSERT INTO hdfs_test SELECT number + 5 FROM numbers(3)")
|
||||
assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n"
|
||||
assert_objects_count(cluster, files_overhead_per_insert * 2 + files_overhead)
|
||||
|
||||
node.query("INSERT INTO hdfs_test SELECT number + 8 FROM numbers(1)")
|
||||
assert node.query("SELECT * FROM hdfs_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n8\n"
|
||||
assert_objects_count(cluster, files_overhead_per_insert * 3 + files_overhead)
|
||||
|
||||
node.query("TRUNCATE TABLE hdfs_test")
|
||||
assert_objects_count(cluster, 0)
|
||||
|
||||
node.query("DROP TABLE hdfs_test")
|
||||
|
0
tests/integration/test_merge_tree_hdfs/__init__.py
Normal file
0
tests/integration/test_merge_tree_hdfs/__init__.py
Normal file
@ -0,0 +1,12 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
</yandex>
|
@ -0,0 +1,30 @@
|
||||
<yandex>
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<hdfs>
|
||||
<type>hdfs</type>
|
||||
<endpoint>hdfs://hdfs1:9000/clickhouse/</endpoint>
|
||||
</hdfs>
|
||||
<hdd>
|
||||
<type>local</type>
|
||||
<path>/</path>
|
||||
</hdd>
|
||||
</disks>
|
||||
<policies>
|
||||
<hdfs>
|
||||
<volumes>
|
||||
<main>
|
||||
<disk>hdfs</disk>
|
||||
</main>
|
||||
<external>
|
||||
<disk>hdd</disk>
|
||||
</external>
|
||||
</volumes>
|
||||
</hdfs>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
<merge_tree>
|
||||
<min_bytes_for_wide_part>0</min_bytes_for_wide_part>
|
||||
</merge_tree>
|
||||
</yandex>
|
317
tests/integration/test_merge_tree_hdfs/test.py
Normal file
317
tests/integration/test_merge_tree_hdfs/test.py
Normal file
@ -0,0 +1,317 @@
|
||||
import logging
|
||||
import random
|
||||
import string
|
||||
import time
|
||||
import threading
|
||||
import os
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node/configs/config.d/storage_conf.xml')
|
||||
|
||||
|
||||
def create_table(cluster, table_name, additional_settings=None):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
create_table_statement = """
|
||||
CREATE TABLE {} (
|
||||
dt Date, id Int64, data String,
|
||||
INDEX min_max (id) TYPE minmax GRANULARITY 3
|
||||
) ENGINE=MergeTree()
|
||||
PARTITION BY dt
|
||||
ORDER BY (dt, id)
|
||||
SETTINGS
|
||||
storage_policy='hdfs',
|
||||
old_parts_lifetime=0,
|
||||
index_granularity=512
|
||||
""".format(table_name)
|
||||
|
||||
if additional_settings:
|
||||
create_table_statement += ","
|
||||
create_table_statement += additional_settings
|
||||
|
||||
node.query(create_table_statement)
|
||||
|
||||
|
||||
FILES_OVERHEAD = 1
|
||||
FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files
|
||||
FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1
|
||||
FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1
|
||||
|
||||
|
||||
def random_string(length):
|
||||
letters = string.ascii_letters
|
||||
return ''.join(random.choice(letters) for i in range(length))
|
||||
|
||||
|
||||
def generate_values(date_str, count, sign=1):
|
||||
data = [[date_str, sign * (i + 1), random_string(10)] for i in range(count)]
|
||||
data.sort(key=lambda tup: tup[1])
|
||||
return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data])
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster():
|
||||
try:
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml",
|
||||
"configs/config.d/log_conf.xml"], with_hdfs=True)
|
||||
logging.info("Starting cluster...")
|
||||
cluster.start()
|
||||
logging.info("Cluster started")
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
fs.mkdirs('/clickhouse')
|
||||
|
||||
logging.info("Created HDFS directory")
|
||||
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def wait_for_delete_hdfs_objects(cluster, expected, num_tries=30):
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
while num_tries > 0:
|
||||
num_hdfs_objects = len(fs.listdir('/clickhouse'))
|
||||
if num_hdfs_objects == expected:
|
||||
break;
|
||||
num_tries -= 1
|
||||
time.sleep(1)
|
||||
assert(len(fs.listdir('/clickhouse')) == expected)
|
||||
|
||||
|
||||
@pytest.fixture(autouse=True)
|
||||
def drop_table(cluster):
|
||||
node = cluster.instances["node"]
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
print('Number of hdfs objects to delete:', len(hdfs_objects), sep=' ')
|
||||
|
||||
node.query("DROP TABLE IF EXISTS hdfs_test SYNC")
|
||||
|
||||
try:
|
||||
wait_for_delete_hdfs_objects(cluster, 0)
|
||||
finally:
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
if len(hdfs_objects) == 0:
|
||||
return
|
||||
print("Manually removing extra objects to prevent tests cascade failing: ", hdfs_objects)
|
||||
for path in hdfs_objects:
|
||||
fs.delete(path)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("min_rows_for_wide_part,files_per_part", [(0, FILES_OVERHEAD_PER_PART_WIDE), (8192, FILES_OVERHEAD_PER_PART_COMPACT)])
|
||||
def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part):
|
||||
create_table(cluster, "hdfs_test", additional_settings="min_rows_for_wide_part={}".format(min_rows_for_wide_part))
|
||||
|
||||
node = cluster.instances["node"]
|
||||
|
||||
values1 = generate_values('2020-01-03', 4096)
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(values1))
|
||||
assert node.query("SELECT * FROM hdfs_test order by dt, id FORMAT Values") == values1
|
||||
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
print(hdfs_objects)
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + files_per_part
|
||||
|
||||
values2 = generate_values('2020-01-04', 4096)
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(values2))
|
||||
assert node.query("SELECT * FROM hdfs_test ORDER BY dt, id FORMAT Values") == values1 + "," + values2
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + files_per_part * 2
|
||||
|
||||
assert node.query("SELECT count(*) FROM hdfs_test where id = 1 FORMAT Values") == "(2)"
|
||||
|
||||
|
||||
def test_alter_table_columns(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096, -1)))
|
||||
|
||||
node.query("ALTER TABLE hdfs_test ADD COLUMN col1 UInt64 DEFAULT 1")
|
||||
# To ensure parts have merged
|
||||
node.query("OPTIMIZE TABLE hdfs_test")
|
||||
|
||||
assert node.query("SELECT sum(col1) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
assert node.query("SELECT sum(col1) FROM hdfs_test WHERE id > 0 FORMAT Values") == "(4096)"
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD_PER_COLUMN)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MODIFY COLUMN col1 String", settings={"mutations_sync": 2})
|
||||
|
||||
assert node.query("SELECT distinct(col1) FROM hdfs_test FORMAT Values") == "('1')"
|
||||
# and file with mutation
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD_PER_COLUMN + 1)
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DROP COLUMN col1", settings={"mutations_sync": 2})
|
||||
|
||||
# and 2 files with mutations
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + 2)
|
||||
|
||||
|
||||
def test_attach_detach_partition(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DETACH PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("ALTER TABLE hdfs_test ATTACH PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DROP PARTITION '2020-01-03'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE
|
||||
|
||||
node.query("ALTER TABLE hdfs_test DETACH PARTITION '2020-01-04'")
|
||||
node.query("ALTER TABLE hdfs_test DROP DETACHED PARTITION '2020-01-04'", settings={"allow_drop_detached": 1})
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD
|
||||
|
||||
|
||||
def test_move_partition_to_another_disk(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-04' TO DISK 'hdd'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-04' TO DISK 'hdfs'")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
|
||||
def test_table_manipulations(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
|
||||
node.query("RENAME TABLE hdfs_test TO hdfs_renamed")
|
||||
assert node.query("SELECT count(*) FROM hdfs_renamed FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("RENAME TABLE hdfs_renamed TO hdfs_test")
|
||||
assert node.query("CHECK TABLE hdfs_test FORMAT Values") == "(1)"
|
||||
|
||||
node.query("DETACH TABLE hdfs_test")
|
||||
node.query("ATTACH TABLE hdfs_test")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2
|
||||
|
||||
node.query("TRUNCATE TABLE hdfs_test")
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD
|
||||
|
||||
|
||||
def test_move_replace_partition_to_another_table(cluster):
|
||||
create_table(cluster, "hdfs_test")
|
||||
|
||||
node = cluster.instances["node"]
|
||||
fs = HdfsClient(hosts='localhost')
|
||||
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-04', 4096)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-05', 4096, -1)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-06', 4096, -1)))
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
||||
create_table(cluster, "hdfs_clone")
|
||||
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-03' TO TABLE hdfs_clone")
|
||||
node.query("ALTER TABLE hdfs_test MOVE PARTITION '2020-01-05' TO TABLE hdfs_clone")
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)"
|
||||
assert node.query("SELECT sum(id) FROM hdfs_clone FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_clone FORMAT Values") == "(8192)"
|
||||
|
||||
# Number of objects in HDFS should be unchanged.
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
||||
# Add new partitions to source table, but with different values and replace them from copied table.
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-03', 4096, -1)))
|
||||
node.query("INSERT INTO hdfs_test VALUES {}".format(generate_values('2020-01-05', 4096)))
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 6
|
||||
|
||||
node.query("ALTER TABLE hdfs_test REPLACE PARTITION '2020-01-03' FROM hdfs_clone")
|
||||
node.query("ALTER TABLE hdfs_test REPLACE PARTITION '2020-01-05' FROM hdfs_clone")
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
assert node.query("SELECT sum(id) FROM hdfs_clone FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_clone FORMAT Values") == "(8192)"
|
||||
|
||||
# Wait for outdated partitions deletion.
|
||||
print(1)
|
||||
wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4)
|
||||
|
||||
node.query("DROP TABLE hdfs_clone NO DELAY")
|
||||
assert node.query("SELECT sum(id) FROM hdfs_test FORMAT Values") == "(0)"
|
||||
assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(16384)"
|
||||
|
||||
# Data should remain in hdfs
|
||||
hdfs_objects = fs.listdir('/clickhouse')
|
||||
assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4
|
||||
|
@ -1,4 +1,4 @@
|
||||
<test max_ignored_relative_change="0.7">
|
||||
<test max_ignored_relative_change="2">
|
||||
<create_query>CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory</create_query>
|
||||
|
||||
<settings>
|
||||
|
File diff suppressed because one or more lines are too long
@ -0,0 +1,4 @@
|
||||
1 2020-11-11
|
||||
1 2021-01-01
|
||||
1 2020-11-11
|
||||
1 2021-01-01
|
11
tests/queries/0_stateless/01881_to_week_monotonic_fix.sql
Normal file
11
tests/queries/0_stateless/01881_to_week_monotonic_fix.sql
Normal file
@ -0,0 +1,11 @@
|
||||
drop table if exists test_tbl;
|
||||
|
||||
create table test_tbl (vend_nm String, ship_dt Date) engine MergeTree partition by toWeek(ship_dt) order by vend_nm;
|
||||
|
||||
insert into test_tbl values('1', '2020-11-11'), ('1', '2021-01-01');
|
||||
|
||||
select * From test_tbl where ship_dt >= toDate('2020-11-01') and ship_dt <= toDate('2021-05-05') order by ship_dt;
|
||||
|
||||
select * From test_tbl where ship_dt >= toDate('2020-01-01') and ship_dt <= toDate('2021-05-05') order by ship_dt;
|
||||
|
||||
drop table test_tbl;
|
@ -236,4 +236,5 @@
|
||||
01801_s3_distributed
|
||||
01833_test_collation_alvarotuso
|
||||
01850_dist_INSERT_preserve_error
|
||||
01870_modulo_partition_key
|
||||
01880_remote_ipv6
|
||||
|
118
tests/testflows/extended_precision_data_types/common.py
Normal file
118
tests/testflows/extended_precision_data_types/common.py
Normal file
@ -0,0 +1,118 @@
|
||||
import uuid
|
||||
|
||||
from contextlib import contextmanager
|
||||
|
||||
from testflows.core.name import basename, parentname
|
||||
from testflows._core.testtype import TestSubType
|
||||
from testflows.asserts import values, error, snapshot
|
||||
|
||||
from helpers.common import *
|
||||
|
||||
@contextmanager
|
||||
def allow_experimental_bigint(node):
|
||||
"""Enable experimental big int setting in Clickhouse.
|
||||
"""
|
||||
setting = ("allow_experimental_bigint_types", 1)
|
||||
default_query_settings = None
|
||||
|
||||
try:
|
||||
with Given("I add allow_experimental_bigint to the default query settings"):
|
||||
default_query_settings = getsattr(current().context, "default_query_settings", [])
|
||||
default_query_settings.append(setting)
|
||||
yield
|
||||
finally:
|
||||
with Finally("I remove allow_experimental_bigint from the default query settings"):
|
||||
if default_query_settings:
|
||||
try:
|
||||
default_query_settings.pop(default_query_settings.index(setting))
|
||||
except ValueError:
|
||||
pass
|
||||
|
||||
@TestStep(Given)
|
||||
def allow_experimental_map_type(self):
|
||||
"""Set allow_experimental_map_type = 1
|
||||
"""
|
||||
setting = ("allow_experimental_map_type", 1)
|
||||
default_query_settings = None
|
||||
|
||||
try:
|
||||
with By("adding allow_experimental_map_type to the default query settings"):
|
||||
default_query_settings = getsattr(current().context, "default_query_settings", [])
|
||||
default_query_settings.append(setting)
|
||||
yield
|
||||
finally:
|
||||
with Finally("I remove allow_experimental_map_type from the default query settings"):
|
||||
if default_query_settings:
|
||||
try:
|
||||
default_query_settings.pop(default_query_settings.index(setting))
|
||||
except ValueError:
|
||||
pass
|
||||
|
||||
def execute_query(sql, expected=None, format="TabSeparatedWithNames", compare_func=None):
|
||||
"""Execute SQL query and compare the output to the snapshot.
|
||||
"""
|
||||
name = basename(current().name)
|
||||
|
||||
with When("I execute query", description=sql):
|
||||
r = current().context.node.query(sql + " FORMAT " + format)
|
||||
|
||||
if expected is not None:
|
||||
with Then("I check output against expected"):
|
||||
|
||||
if compare_func is None:
|
||||
assert r.output.strip() == expected, error()
|
||||
|
||||
else:
|
||||
assert compare_func(r.output.strip(), expected), error()
|
||||
|
||||
else:
|
||||
with Then("I check output against snapshot"):
|
||||
with values() as that:
|
||||
assert that(snapshot("\n" + r.output.strip() + "\n", "tests", name=name, encoder=str)), error()
|
||||
|
||||
@TestStep(Given)
|
||||
def table(self, data_type, name="table0"):
|
||||
"""Create a table.
|
||||
"""
|
||||
node = current().context.node
|
||||
|
||||
try:
|
||||
with By("creating table"):
|
||||
node.query(f"CREATE TABLE {name}(a {data_type}) ENGINE = Memory")
|
||||
yield
|
||||
|
||||
finally:
|
||||
with Finally("drop the table"):
|
||||
node.query(f"DROP TABLE IF EXISTS {name}")
|
||||
|
||||
def getuid():
|
||||
"""Create a unique variable name based on the test it is called from.
|
||||
"""
|
||||
if current().subtype == TestSubType.Example:
|
||||
testname = f"{basename(parentname(current().name)).replace(' ', '_').replace(',','')}"
|
||||
else:
|
||||
testname = f"{basename(current().name).replace(' ', '_').replace(',','')}"
|
||||
|
||||
for char in ['(', ')', '[', ']','\'']:
|
||||
testname = testname.replace(f'{char}', '')
|
||||
|
||||
return testname + "_" + str(uuid.uuid1()).replace('-', '_')
|
||||
|
||||
def to_data_type(data_type, value):
|
||||
"""Return a conversion statement based on the data type provided
|
||||
"""
|
||||
if data_type in ['Decimal256(0)']:
|
||||
return f'toDecimal256(\'{value}\',0)'
|
||||
|
||||
else:
|
||||
return f'to{data_type}(\'{value}\')'
|
||||
|
||||
|
||||
data_types = [
|
||||
('Int128', '-170141183460469231731687303715884105728', '170141183460469231731687303715884105727'),
|
||||
('Int256', '-57896044618658097711785492504343953926634992332820282019728792003956564819968', '57896044618658097711785492504343953926634992332820282019728792003956564819967'),
|
||||
('UInt128','0','340282366920938463463374607431768211455'),
|
||||
('UInt256', '0', '115792089237316195423570985008687907853269984665640564039457584007913129639935'),
|
||||
]
|
||||
|
||||
Decimal256_min_max = -1000000000000000000000000000000000000000000000000000000000000000000000000000,1000000000000000000000000000000000000000000000000000000000000000000000000000
|
@ -0,0 +1,6 @@
|
||||
<yandex>
|
||||
<timezone>Europe/Moscow</timezone>
|
||||
<listen_host replace="replace">::</listen_host>
|
||||
<path>/var/lib/clickhouse/</path>
|
||||
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
|
||||
</yandex>
|
@ -0,0 +1,17 @@
|
||||
<yandex>
|
||||
<shutdown_wait_unfinished>3</shutdown_wait_unfinished>
|
||||
<logger>
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/log.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
|
||||
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
|
||||
</logger>
|
||||
<part_log>
|
||||
<database>system</database>
|
||||
<table>part_log</table>
|
||||
<flush_interval_milliseconds>500</flush_interval_milliseconds>
|
||||
</part_log>
|
||||
</yandex>
|
@ -0,0 +1,20 @@
|
||||
<yandex>
|
||||
|
||||
<storage_configuration>
|
||||
<disks>
|
||||
<default>
|
||||
<keep_free_space_bytes>1024</keep_free_space_bytes>
|
||||
</default>
|
||||
</disks>
|
||||
<policies>
|
||||
<default>
|
||||
<volumes>
|
||||
<default>
|
||||
<disk>default</disk>
|
||||
</default>
|
||||
</volumes>
|
||||
</default>
|
||||
</policies>
|
||||
</storage_configuration>
|
||||
|
||||
</yandex>
|
@ -0,0 +1,448 @@
|
||||
<?xml version="1.0"?>
|
||||
<!--
|
||||
NOTE: User and query level settings are set up in "users.xml" file.
|
||||
-->
|
||||
<yandex>
|
||||
<logger>
|
||||
<!-- Possible levels: https://github.com/pocoproject/poco/blob/develop/Foundation/include/Poco/Logger.h#L105 -->
|
||||
<level>trace</level>
|
||||
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
|
||||
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
|
||||
<size>1000M</size>
|
||||
<count>10</count>
|
||||
<!-- <console>1</console> --> <!-- Default behavior is autodetection (log to console if not daemon mode and is tty) -->
|
||||
</logger>
|
||||
<!--display_name>production</display_name--> <!-- It is the name that will be shown in the client -->
|
||||
<http_port>8123</http_port>
|
||||
<tcp_port>9000</tcp_port>
|
||||
|
||||
<!-- For HTTPS and SSL over native protocol. -->
|
||||
<!--
|
||||
<https_port>8443</https_port>
|
||||
<tcp_port_secure>9440</tcp_port_secure>
|
||||
-->
|
||||
|
||||
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
|
||||
<openSSL>
|
||||
<server> <!-- Used for https server AND secure tcp port -->
|
||||
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
|
||||
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
|
||||
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
|
||||
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
</server>
|
||||
|
||||
<client> <!-- Used for connecting to https dictionary source -->
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
<!-- Use for self-signed: <verificationMode>none</verificationMode> -->
|
||||
<invalidCertificateHandler>
|
||||
<!-- Use for self-signed: <name>AcceptCertificateHandler</name> -->
|
||||
<name>RejectCertificateHandler</name>
|
||||
</invalidCertificateHandler>
|
||||
</client>
|
||||
</openSSL>
|
||||
|
||||
<!-- Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 -->
|
||||
<!--
|
||||
<http_server_default_response><![CDATA[<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>]]></http_server_default_response>
|
||||
-->
|
||||
|
||||
<!-- Port for communication between replicas. Used for data exchange. -->
|
||||
<interserver_http_port>9009</interserver_http_port>
|
||||
|
||||
<!-- Hostname that is used by other replicas to request this server.
|
||||
If not specified, than it is determined analoguous to 'hostname -f' command.
|
||||
This setting could be used to switch replication to another network interface.
|
||||
-->
|
||||
<!--
|
||||
<interserver_http_host>example.yandex.ru</interserver_http_host>
|
||||
-->
|
||||
|
||||
<!-- Listen specified host. use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. -->
|
||||
<!-- <listen_host>::</listen_host> -->
|
||||
<!-- Same for hosts with disabled ipv6: -->
|
||||
<listen_host>0.0.0.0</listen_host>
|
||||
|
||||
<!-- Default values - try listen localhost on ipv4 and ipv6: -->
|
||||
<!--
|
||||
<listen_host>::1</listen_host>
|
||||
<listen_host>127.0.0.1</listen_host>
|
||||
-->
|
||||
<!-- Don't exit if ipv6 or ipv4 unavailable, but listen_host with this protocol specified -->
|
||||
<!-- <listen_try>0</listen_try> -->
|
||||
|
||||
<!-- Allow listen on same address:port -->
|
||||
<!-- <listen_reuse_port>0</listen_reuse_port> -->
|
||||
|
||||
<!-- <listen_backlog>64</listen_backlog> -->
|
||||
|
||||
<max_connections>4096</max_connections>
|
||||
<keep_alive_timeout>3</keep_alive_timeout>
|
||||
|
||||
<!-- Maximum number of concurrent queries. -->
|
||||
<max_concurrent_queries>100</max_concurrent_queries>
|
||||
|
||||
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
|
||||
correct maximum value. -->
|
||||
<!-- <max_open_files>262144</max_open_files> -->
|
||||
|
||||
<!-- Size of cache of uncompressed blocks of data, used in tables of MergeTree family.
|
||||
In bytes. Cache is single for server. Memory is allocated only on demand.
|
||||
Cache is used when 'use_uncompressed_cache' user setting turned on (off by default).
|
||||
Uncompressed cache is advantageous only for very short queries and in rare cases.
|
||||
-->
|
||||
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
|
||||
|
||||
<!-- Approximate size of mark cache, used in tables of MergeTree family.
|
||||
In bytes. Cache is single for server. Memory is allocated only on demand.
|
||||
You should not lower this value.
|
||||
-->
|
||||
<mark_cache_size>5368709120</mark_cache_size>
|
||||
|
||||
|
||||
<!-- Path to data directory, with trailing slash. -->
|
||||
<path>/var/lib/clickhouse/</path>
|
||||
|
||||
<!-- Path to temporary data for processing hard queries. -->
|
||||
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
|
||||
|
||||
<!-- Directory with user provided files that are accessible by 'file' table function. -->
|
||||
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
|
||||
|
||||
<!-- Path to folder where users and roles created by SQL commands are stored. -->
|
||||
<access_control_path>/var/lib/clickhouse/access/</access_control_path>
|
||||
|
||||
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
|
||||
<user_directories>
|
||||
<users_xml>
|
||||
<!-- Path to configuration file with predefined users. -->
|
||||
<path>users.xml</path>
|
||||
</users_xml>
|
||||
<local_directory>
|
||||
<!-- Path to folder where users created by SQL commands are stored. -->
|
||||
<path>/var/lib/clickhouse/access/</path>
|
||||
</local_directory>
|
||||
</user_directories>
|
||||
|
||||
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
|
||||
<users_config>users.xml</users_config>
|
||||
|
||||
<!-- Default profile of settings. -->
|
||||
<default_profile>default</default_profile>
|
||||
|
||||
<!-- System profile of settings. This settings are used by internal processes (Buffer storage, Distibuted DDL worker and so on). -->
|
||||
<!-- <system_profile>default</system_profile> -->
|
||||
|
||||
<!-- Default database. -->
|
||||
<default_database>default</default_database>
|
||||
|
||||
<!-- Server time zone could be set here.
|
||||
|
||||
Time zone is used when converting between String and DateTime types,
|
||||
when printing DateTime in text formats and parsing DateTime from text,
|
||||
it is used in date and time related functions, if specific time zone was not passed as an argument.
|
||||
|
||||
Time zone is specified as identifier from IANA time zone database, like UTC or Africa/Abidjan.
|
||||
If not specified, system time zone at server startup is used.
|
||||
|
||||
Please note, that server could display time zone alias instead of specified name.
|
||||
Example: W-SU is an alias for Europe/Moscow and Zulu is an alias for UTC.
|
||||
-->
|
||||
<!-- <timezone>Europe/Moscow</timezone> -->
|
||||
|
||||
<!-- You can specify umask here (see "man umask"). Server will apply it on startup.
|
||||
Number is always parsed as octal. Default umask is 027 (other users cannot read logs, data files, etc; group can only read).
|
||||
-->
|
||||
<!-- <umask>022</umask> -->
|
||||
|
||||
<!-- Perform mlockall after startup to lower first queries latency
|
||||
and to prevent clickhouse executable from being paged out under high IO load.
|
||||
Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
|
||||
-->
|
||||
<mlock_executable>false</mlock_executable>
|
||||
|
||||
<!-- Configuration of clusters that could be used in Distributed tables.
|
||||
https://clickhouse.yandex/docs/en/table_engines/distributed/
|
||||
-->
|
||||
<remote_servers incl="clickhouse_remote_servers" >
|
||||
<!-- Test only shard config for testing distributed storage -->
|
||||
<test_shard_localhost>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost>
|
||||
<test_cluster_two_shards_localhost>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_cluster_two_shards_localhost>
|
||||
<test_shard_localhost_secure>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9440</port>
|
||||
<secure>1</secure>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost_secure>
|
||||
<test_unavailable_shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>1</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_unavailable_shard>
|
||||
</remote_servers>
|
||||
|
||||
|
||||
<!-- If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file.
|
||||
By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element.
|
||||
Values for substitutions are specified in /yandex/name_of_substitution elements in that file.
|
||||
-->
|
||||
|
||||
<!-- ZooKeeper is used to store metadata about replicas, when using Replicated tables.
|
||||
Optional. If you don't use replicated tables, you could omit that.
|
||||
|
||||
See https://clickhouse.yandex/docs/en/table_engines/replication/
|
||||
-->
|
||||
<zookeeper incl="zookeeper-servers" optional="true" />
|
||||
|
||||
<!-- Substitutions for parameters of replicated tables.
|
||||
Optional. If you don't use replicated tables, you could omit that.
|
||||
|
||||
See https://clickhouse.yandex/docs/en/table_engines/replication/#creating-replicated-tables
|
||||
-->
|
||||
<macros incl="macros" optional="true" />
|
||||
|
||||
|
||||
<!-- Reloading interval for embedded dictionaries, in seconds. Default: 3600. -->
|
||||
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
|
||||
|
||||
|
||||
<!-- Maximum session timeout, in seconds. Default: 3600. -->
|
||||
<max_session_timeout>3600</max_session_timeout>
|
||||
|
||||
<!-- Default session timeout, in seconds. Default: 60. -->
|
||||
<default_session_timeout>60</default_session_timeout>
|
||||
|
||||
<!-- Sending data to Graphite for monitoring. Several sections can be defined. -->
|
||||
<!--
|
||||
interval - send every X second
|
||||
root_path - prefix for keys
|
||||
hostname_in_path - append hostname to root_path (default = true)
|
||||
metrics - send data from table system.metrics
|
||||
events - send data from table system.events
|
||||
asynchronous_metrics - send data from table system.asynchronous_metrics
|
||||
-->
|
||||
<!--
|
||||
<graphite>
|
||||
<host>localhost</host>
|
||||
<port>42000</port>
|
||||
<timeout>0.1</timeout>
|
||||
<interval>60</interval>
|
||||
<root_path>one_min</root_path>
|
||||
<hostname_in_path>true</hostname_in_path>
|
||||
|
||||
<metrics>true</metrics>
|
||||
<events>true</events>
|
||||
<asynchronous_metrics>true</asynchronous_metrics>
|
||||
</graphite>
|
||||
<graphite>
|
||||
<host>localhost</host>
|
||||
<port>42000</port>
|
||||
<timeout>0.1</timeout>
|
||||
<interval>1</interval>
|
||||
<root_path>one_sec</root_path>
|
||||
|
||||
<metrics>true</metrics>
|
||||
<events>true</events>
|
||||
<asynchronous_metrics>false</asynchronous_metrics>
|
||||
</graphite>
|
||||
-->
|
||||
|
||||
|
||||
<!-- Query log. Used only for queries with setting log_queries = 1. -->
|
||||
<query_log>
|
||||
<!-- What table to insert data. If table is not exist, it will be created.
|
||||
When query log structure is changed after system update,
|
||||
then old table will be renamed and new table will be created automatically.
|
||||
-->
|
||||
<database>system</database>
|
||||
<table>query_log</table>
|
||||
<!--
|
||||
PARTITION BY expr https://clickhouse.yandex/docs/en/table_engines/custom_partitioning_key/
|
||||
Example:
|
||||
event_date
|
||||
toMonday(event_date)
|
||||
toYYYYMM(event_date)
|
||||
toStartOfHour(event_time)
|
||||
-->
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<!-- Interval of flushing data. -->
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</query_log>
|
||||
|
||||
<!-- Trace log. Stores stack traces collected by query profilers.
|
||||
See query_profiler_real_time_period_ns and query_profiler_cpu_time_period_ns settings. -->
|
||||
<trace_log>
|
||||
<database>system</database>
|
||||
<table>trace_log</table>
|
||||
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</trace_log>
|
||||
|
||||
<!-- Query thread log. Has information about all threads participated in query execution.
|
||||
Used only for queries with setting log_query_threads = 1. -->
|
||||
<query_thread_log>
|
||||
<database>system</database>
|
||||
<table>query_thread_log</table>
|
||||
<partition_by>toYYYYMM(event_date)</partition_by>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</query_thread_log>
|
||||
|
||||
<!-- Uncomment if use part log.
|
||||
Part log contains information about all actions with parts in MergeTree tables (creation, deletion, merges, downloads).
|
||||
<part_log>
|
||||
<database>system</database>
|
||||
<table>part_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</part_log>
|
||||
-->
|
||||
|
||||
<!-- Uncomment to write text log into table.
|
||||
Text log contains all information from usual server log but stores it in structured and efficient way.
|
||||
<text_log>
|
||||
<database>system</database>
|
||||
<table>text_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
</text_log>
|
||||
-->
|
||||
|
||||
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
|
||||
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/
|
||||
-->
|
||||
|
||||
<!-- Path to file with region hierarchy. -->
|
||||
<!-- <path_to_regions_hierarchy_file>/opt/geo/regions_hierarchy.txt</path_to_regions_hierarchy_file> -->
|
||||
|
||||
<!-- Path to directory with files containing names of regions -->
|
||||
<!-- <path_to_regions_names_files>/opt/geo/</path_to_regions_names_files> -->
|
||||
|
||||
|
||||
<!-- Configuration of external dictionaries. See:
|
||||
https://clickhouse.yandex/docs/en/dicts/external_dicts/
|
||||
-->
|
||||
<dictionaries_config>*_dictionary.xml</dictionaries_config>
|
||||
|
||||
<!-- Uncomment if you want data to be compressed 30-100% better.
|
||||
Don't do that if you just started using ClickHouse.
|
||||
-->
|
||||
<compression incl="clickhouse_compression">
|
||||
<!--
|
||||
<!- - Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - ->
|
||||
<case>
|
||||
|
||||
<!- - Conditions. All must be satisfied. Some conditions may be omitted. - ->
|
||||
<min_part_size>10000000000</min_part_size> <!- - Min part size in bytes. - ->
|
||||
<min_part_size_ratio>0.01</min_part_size_ratio> <!- - Min size of part relative to whole table size. - ->
|
||||
|
||||
<!- - What compression method to use. - ->
|
||||
<method>zstd</method>
|
||||
</case>
|
||||
-->
|
||||
</compression>
|
||||
|
||||
<!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster.
|
||||
Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. -->
|
||||
<distributed_ddl>
|
||||
<!-- Path in ZooKeeper to queue with DDL queries -->
|
||||
<path>/clickhouse/task_queue/ddl</path>
|
||||
|
||||
<!-- Settings from this profile will be used to execute DDL queries -->
|
||||
<!-- <profile>default</profile> -->
|
||||
</distributed_ddl>
|
||||
|
||||
<!-- Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
|
||||
<!--
|
||||
<merge_tree>
|
||||
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
|
||||
</merge_tree>
|
||||
-->
|
||||
|
||||
<!-- Protection from accidental DROP.
|
||||
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
|
||||
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
|
||||
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
|
||||
The same for max_partition_size_to_drop.
|
||||
Uncomment to disable protection.
|
||||
-->
|
||||
<!-- <max_table_size_to_drop>0</max_table_size_to_drop> -->
|
||||
<!-- <max_partition_size_to_drop>0</max_partition_size_to_drop> -->
|
||||
|
||||
<!-- Example of parameters for GraphiteMergeTree table engine -->
|
||||
<graphite_rollup_example>
|
||||
<pattern>
|
||||
<regexp>click_cost</regexp>
|
||||
<function>any</function>
|
||||
<retention>
|
||||
<age>0</age>
|
||||
<precision>3600</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>86400</age>
|
||||
<precision>60</precision>
|
||||
</retention>
|
||||
</pattern>
|
||||
<default>
|
||||
<function>max</function>
|
||||
<retention>
|
||||
<age>0</age>
|
||||
<precision>60</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>3600</age>
|
||||
<precision>300</precision>
|
||||
</retention>
|
||||
<retention>
|
||||
<age>86400</age>
|
||||
<precision>3600</precision>
|
||||
</retention>
|
||||
</default>
|
||||
</graphite_rollup_example>
|
||||
|
||||
<!-- Directory in <clickhouse-path> containing schema files for various input formats.
|
||||
The directory will be created if it doesn't exist.
|
||||
-->
|
||||
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
|
||||
|
||||
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
|
||||
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
|
||||
</yandex>
|
@ -0,0 +1,133 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<!-- Profiles of settings. -->
|
||||
<profiles>
|
||||
<!-- Default settings. -->
|
||||
<default>
|
||||
<!-- Maximum memory usage for processing single query, in bytes. -->
|
||||
<max_memory_usage>10000000000</max_memory_usage>
|
||||
|
||||
<!-- Use cache of uncompressed blocks of data. Meaningful only for processing many of very short queries. -->
|
||||
<use_uncompressed_cache>0</use_uncompressed_cache>
|
||||
|
||||
<!-- How to choose between replicas during distributed query processing.
|
||||
random - choose random replica from set of replicas with minimum number of errors
|
||||
nearest_hostname - from set of replicas with minimum number of errors, choose replica
|
||||
with minimum number of different symbols between replica's hostname and local hostname
|
||||
(Hamming distance).
|
||||
in_order - first live replica is chosen in specified order.
|
||||
first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors.
|
||||
-->
|
||||
<load_balancing>random</load_balancing>
|
||||
</default>
|
||||
|
||||
<!-- Profile that allows only read queries. -->
|
||||
<readonly>
|
||||
<readonly>1</readonly>
|
||||
</readonly>
|
||||
</profiles>
|
||||
|
||||
<!-- Users and ACL. -->
|
||||
<users>
|
||||
<!-- If user name was not specified, 'default' user is used. -->
|
||||
<default>
|
||||
<!-- Password could be specified in plaintext or in SHA256 (in hex format).
|
||||
|
||||
If you want to specify password in plaintext (not recommended), place it in 'password' element.
|
||||
Example: <password>qwerty</password>.
|
||||
Password could be empty.
|
||||
|
||||
If you want to specify SHA256, place it in 'password_sha256_hex' element.
|
||||
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
|
||||
Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019).
|
||||
|
||||
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
|
||||
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
|
||||
|
||||
How to generate decent password:
|
||||
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
|
||||
In first line will be password and in second - corresponding SHA256.
|
||||
|
||||
How to generate double SHA1:
|
||||
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1
|
||||
In first line will be password and in second - corresponding double SHA1.
|
||||
-->
|
||||
<password></password>
|
||||
|
||||
<!-- List of networks with open access.
|
||||
|
||||
To open access from everywhere, specify:
|
||||
<ip>::/0</ip>
|
||||
|
||||
To open access only from localhost, specify:
|
||||
<ip>::1</ip>
|
||||
<ip>127.0.0.1</ip>
|
||||
|
||||
Each element of list has one of the following forms:
|
||||
<ip> IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0
|
||||
2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::.
|
||||
<host> Hostname. Example: server01.yandex.ru.
|
||||
To check access, DNS query is performed, and all received addresses compared to peer address.
|
||||
<host_regexp> Regular expression for host names. Example, ^server\d\d-\d\d-\d\.yandex\.ru$
|
||||
To check access, DNS PTR query is performed for peer address and then regexp is applied.
|
||||
Then, for result of PTR query, another DNS query is performed and all received addresses compared to peer address.
|
||||
Strongly recommended that regexp is ends with $
|
||||
All results of DNS requests are cached till server restart.
|
||||
-->
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::/0</ip>
|
||||
</networks>
|
||||
|
||||
<!-- Settings profile for user. -->
|
||||
<profile>default</profile>
|
||||
|
||||
<!-- Quota for user. -->
|
||||
<quota>default</quota>
|
||||
|
||||
<!-- Allow access management -->
|
||||
<access_management>1</access_management>
|
||||
|
||||
<!-- Example of row level security policy. -->
|
||||
<!-- <databases>
|
||||
<test>
|
||||
<filtered_table1>
|
||||
<filter>a = 1</filter>
|
||||
</filtered_table1>
|
||||
<filtered_table2>
|
||||
<filter>a + b < 1 or c - d > 5</filter>
|
||||
</filtered_table2>
|
||||
</test>
|
||||
</databases> -->
|
||||
</default>
|
||||
|
||||
<!-- Example of user with readonly access. -->
|
||||
<!-- <readonly>
|
||||
<password></password>
|
||||
<networks incl="networks" replace="replace">
|
||||
<ip>::1</ip>
|
||||
<ip>127.0.0.1</ip>
|
||||
</networks>
|
||||
<profile>readonly</profile>
|
||||
<quota>default</quota>
|
||||
</readonly> -->
|
||||
</users>
|
||||
|
||||
<!-- Quotas. -->
|
||||
<quotas>
|
||||
<!-- Name of quota. -->
|
||||
<default>
|
||||
<!-- Limits for time interval. You could specify many intervals with different limits. -->
|
||||
<interval>
|
||||
<!-- Length of interval. -->
|
||||
<duration>3600</duration>
|
||||
|
||||
<!-- No limits. Just calculate resource usage for time interval. -->
|
||||
<queries>0</queries>
|
||||
<errors>0</errors>
|
||||
<result_rows>0</result_rows>
|
||||
<read_rows>0</read_rows>
|
||||
<execution_time>0</execution_time>
|
||||
</interval>
|
||||
</default>
|
||||
</quotas>
|
||||
</yandex>
|
11
tests/testflows/extended_precision_data_types/errors.py
Normal file
11
tests/testflows/extended_precision_data_types/errors.py
Normal file
@ -0,0 +1,11 @@
|
||||
def not_implemented_bigints(name):
|
||||
return(48, f"Exception: {name} is not implemented for big integers")
|
||||
|
||||
def bigints_not_implements(name):
|
||||
return(48, f'Exception: {name} for big integers is not implemented')
|
||||
|
||||
def illegal_type():
|
||||
return(43, 'Exception: Illegal type')
|
||||
|
||||
def illegal_column():
|
||||
return(44, 'Exception: Illegal column')
|
@ -0,0 +1,27 @@
|
||||
version: '2.3'
|
||||
|
||||
services:
|
||||
clickhouse:
|
||||
image: yandex/clickhouse-integration-test
|
||||
expose:
|
||||
- "9000"
|
||||
- "9009"
|
||||
- "8123"
|
||||
volumes:
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.d:/etc/clickhouse-server/config.d"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.d:/etc/clickhouse-server/users.d"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.xml:/etc/clickhouse-server/config.xml"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.xml:/etc/clickhouse-server/users.xml"
|
||||
- "${CLICKHOUSE_TESTS_SERVER_BIN_PATH:-/usr/bin/clickhouse}:/usr/bin/clickhouse"
|
||||
- "${CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH:-/usr/bin/clickhouse-odbc-bridge}:/usr/bin/clickhouse-odbc-bridge"
|
||||
entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log"
|
||||
healthcheck:
|
||||
test: clickhouse client --query='select 1'
|
||||
interval: 10s
|
||||
timeout: 10s
|
||||
retries: 3
|
||||
start_period: 300s
|
||||
cap_add:
|
||||
- SYS_PTRACE
|
||||
security_opt:
|
||||
- label:disable
|
@ -0,0 +1,30 @@
|
||||
version: '2.3'
|
||||
|
||||
services:
|
||||
|
||||
mysql1:
|
||||
extends:
|
||||
file: mysql-service.yml
|
||||
service: mysql
|
||||
hostname: mysql1
|
||||
volumes:
|
||||
- "${CLICKHOUSE_TESTS_DIR}/_instances/mysql1/database:/var/lib/mysql"
|
||||
|
||||
clickhouse1:
|
||||
extends:
|
||||
file: clickhouse-service.yml
|
||||
service: clickhouse
|
||||
hostname: clickhouse1
|
||||
volumes:
|
||||
- "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/database/:/var/lib/clickhouse/"
|
||||
- "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/logs/:/var/log/clickhouse-server/"
|
||||
|
||||
# dummy service which does nothing, but allows to postpone
|
||||
# 'docker-compose up -d' till all dependecies will go healthy
|
||||
all_services_ready:
|
||||
image: hello-world
|
||||
depends_on:
|
||||
clickhouse1:
|
||||
condition: service_healthy
|
||||
mysql1:
|
||||
condition: service_healthy
|
@ -0,0 +1,19 @@
|
||||
version: '2.3'
|
||||
|
||||
services:
|
||||
mysql:
|
||||
image: mysql:5.7.30
|
||||
restart: always
|
||||
environment:
|
||||
MYSQL_DATABASE: 'db'
|
||||
MYSQL_USER: 'user'
|
||||
MYSQL_PASSWORD: 'password'
|
||||
MYSQL_ROOT_PASSWORD: 'password'
|
||||
expose:
|
||||
- '3306'
|
||||
healthcheck:
|
||||
test: mysql -D db -u user --password=password -e "select 1;"
|
||||
interval: 3s
|
||||
timeout: 2s
|
||||
retries: 40
|
||||
start_period: 2s
|
52
tests/testflows/extended_precision_data_types/regression.py
Executable file
52
tests/testflows/extended_precision_data_types/regression.py
Executable file
@ -0,0 +1,52 @@
|
||||
#!/usr/bin/env python3
|
||||
import os
|
||||
import sys
|
||||
|
||||
from testflows.core import *
|
||||
|
||||
append_path(sys.path, "..")
|
||||
|
||||
from helpers.cluster import Cluster
|
||||
from helpers.argparser import argparser
|
||||
from extended_precision_data_types.requirements import *
|
||||
|
||||
xfails = {
|
||||
}
|
||||
|
||||
xflags = {
|
||||
}
|
||||
|
||||
@TestModule
|
||||
@ArgumentParser(argparser)
|
||||
@XFails(xfails)
|
||||
@XFlags(xflags)
|
||||
@Name("extended precision data types")
|
||||
@Specifications(
|
||||
QA_SRS020_ClickHouse_Extended_Precision_Data_Types
|
||||
)
|
||||
@Requirements(
|
||||
RQ_SRS_020_ClickHouse_Extended_Precision("1.0"),
|
||||
)
|
||||
def regression(self, local, clickhouse_binary_path, stress=None, parallel=None):
|
||||
"""Extended precision data type regression.
|
||||
"""
|
||||
|
||||
top().terminating = False
|
||||
|
||||
nodes = {
|
||||
"clickhouse":
|
||||
("clickhouse1",)
|
||||
}
|
||||
with Cluster(local, clickhouse_binary_path, nodes=nodes,
|
||||
docker_compose_project_dir=os.path.join(current_dir(), "extended-precision-data-type_env")) as cluster:
|
||||
|
||||
self.context.cluster = cluster
|
||||
self.context.stress = stress
|
||||
|
||||
if parallel is not None:
|
||||
self.context.parallel = parallel
|
||||
|
||||
Feature(run=load("extended_precision_data_types.tests.feature", "feature"))
|
||||
|
||||
if main():
|
||||
regression()
|
@ -0,0 +1 @@
|
||||
from .requirements import *
|
@ -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
@ -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)
|
@ -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)
|
179
tests/testflows/extended_precision_data_types/tests/bit.py
Normal file
179
tests/testflows/extended_precision_data_types/tests/bit.py
Normal 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)
|
@ -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)
|
@ -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)
|
@ -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"))
|
@ -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)
|
@ -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)
|
119
tests/testflows/extended_precision_data_types/tests/null.py
Normal file
119
tests/testflows/extended_precision_data_types/tests/null.py
Normal 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)
|
191
tests/testflows/extended_precision_data_types/tests/rounding.py
Normal file
191
tests/testflows/extended_precision_data_types/tests/rounding.py
Normal 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)
|
35
tests/testflows/extended_precision_data_types/tests/table.py
Normal file
35
tests/testflows/extended_precision_data_types/tests/table.py
Normal 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}")
|
@ -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)
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user