Merge branch 'master' into patch-26

This commit is contained in:
Denny Crane 2023-05-26 10:54:10 -03:00 committed by GitHub
commit 349c92ccaf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 971 additions and 179 deletions

View File

@ -258,4 +258,4 @@ Since [remote](../../../sql-reference/table-functions/remote.md) and [cluster](.
- [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description
- [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting - [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting
- [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions - [shardNum()](../../../sql-reference/functions/other-functions.md#shardnum) and [shardCount()](../../../sql-reference/functions/other-functions.md#shardcount) functions

View File

@ -167,9 +167,9 @@ user = 'myuser',
password = 'mypass', password = 'mypass',
host = '127.0.0.1', host = '127.0.0.1',
port = 3306, port = 3306,
database = 'test' database = 'test',
connection_pool_size = 8 connection_pool_size = 8,
on_duplicate_clause = 1 on_duplicate_clause = 1,
replace_query = 1 replace_query = 1
``` ```

View File

@ -0,0 +1,27 @@
---
slug: /en/operations/system-tables/build_options
---
# build_options
Contains information about the ClickHouse server's build options.
Columns:
- `name` (String) — Name of the build option, e.g. `USE_ODBC`
- `value` (String) — Value of the build option, e.g. `1`
**Example**
``` sql
SELECT * FROM system.build_options LIMIT 5
```
``` text
┌─name─────────────┬─value─┐
│ USE_BROTLI │ 1 │
│ USE_BZIP2 │ 1 │
│ USE_CAPNP │ 1 │
│ USE_CASSANDRA │ 1 │
│ USE_DATASKETCHES │ 1 │
└──────────────────┴───────┘
```

View File

@ -323,11 +323,11 @@ Alias: `REPEAT`
**Arguments** **Arguments**
- `s` — The string to repeat. [String](../../sql-reference/data-types/string.md). - `s` — The string to repeat. [String](../../sql-reference/data-types/string.md).
- `n` — The number of times to repeat the string. [UInt or Int](../../sql-reference/data-types/int-uint.md). - `n` — The number of times to repeat the string. [UInt* or Int*](../../sql-reference/data-types/int-uint.md).
**Returned value** **Returned value**
The single string containing string `s` repeated `n` times. If `n` \< 1, the function returns empty string. A string containing string `s` repeated `n` times. If `n` <= 0, the function returns the empty string.
Type: `String`. Type: `String`.
@ -345,6 +345,44 @@ Result:
└────────────────────────────────┘ └────────────────────────────────┘
``` ```
## space
Concatenates a space (` `) as many times with itself as specified.
**Syntax**
``` sql
space(n)
```
Alias: `SPACE`.
**Arguments**
- `n` — The number of times to repeat the space. [UInt* or Int*](../../sql-reference/data-types/int-uint.md).
**Returned value**
The string containing string ` ` repeated `n` times. If `n` <= 0, the function returns the empty string.
Type: `String`.
**Example**
Query:
``` sql
SELECT space(3);
```
Result:
``` text
┌─space(3) ────┐
│ │
└──────────────┘
```
## reverse ## reverse
Reverses the sequence of bytes in a string. Reverses the sequence of bytes in a string.

View File

@ -544,10 +544,10 @@ Result:
└─────┴──────────┴───────┘ └─────┴──────────┴───────┘
``` ```
##Filling grouped by sorting prefix ## Filling grouped by sorting prefix
It can be useful to fill rows which have the same values in particular columns independently, - a good example is filling missing values in time series. It can be useful to fill rows which have the same values in particular columns independently, - a good example is filling missing values in time series.
Assume there is the following time series table Assume there is the following time series table:
``` sql ``` sql
CREATE TABLE timeseries CREATE TABLE timeseries
( (
@ -567,7 +567,7 @@ SELECT * FROM timeseries;
└───────────┴─────────────────────────┴───────┘ └───────────┴─────────────────────────┴───────┘
``` ```
And we'd like to fill missing values for each sensor independently with 1 second interval. And we'd like to fill missing values for each sensor independently with 1 second interval.
The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp` The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp`:
``` ```
SELECT * SELECT *
FROM timeseries FROM timeseries
@ -589,7 +589,7 @@ INTERPOLATE ( value AS 9999 )
│ 432 │ 2021-12-01 00:00:05.000 │ 5 │ │ 432 │ 2021-12-01 00:00:05.000 │ 5 │
└───────────┴─────────────────────────┴───────┘ └───────────┴─────────────────────────┴───────┘
``` ```
Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable.
This behavior is controlled by setting `use_with_fill_by_sorting_prefix` (enabled by default) This behavior is controlled by setting `use_with_fill_by_sorting_prefix` (enabled by default)
## Related content ## Related content

View File

@ -1041,18 +1041,16 @@ void AsynchronousMetrics::update(TimePoint update_time)
// It doesn't read the EOL itself. // It doesn't read the EOL itself.
++cpuinfo->position(); ++cpuinfo->position();
if (s.rfind("processor", 0) == 0) static constexpr std::string_view PROCESSOR = "processor";
if (s.starts_with(PROCESSOR))
{ {
/// s390x example: processor 0: version = FF, identification = 039C88, machine = 3906 /// s390x example: processor 0: version = FF, identification = 039C88, machine = 3906
/// non s390x example: processor : 0 /// non s390x example: processor : 0
if (auto colon = s.find_first_of(':')) auto core_id_start = std::ssize(PROCESSOR);
{ while (core_id_start < std::ssize(s) && !std::isdigit(s[core_id_start]))
#ifdef __s390x__ ++core_id_start;
core_id = std::stoi(s.substr(10)); /// 10: length of "processor" plus 1
#else core_id = std::stoi(s.substr(core_id_start));
core_id = std::stoi(s.substr(colon + 2));
#endif
}
} }
else if (s.rfind("cpu MHz", 0) == 0) else if (s.rfind("cpu MHz", 0) == 0)
{ {

View File

@ -138,19 +138,6 @@ namespace
} }
} }
String getCurrentKey(const String & path, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(settings.current_key_id);
if (it == settings.keys.end())
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR,
"Not found a key with the current ID {} required to cipher file {}",
settings.current_key_id,
quoteString(path));
return it->second;
}
String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings)
{ {
auto it = settings.keys.find(header.key_id); auto it = settings.keys.find(header.key_id);
@ -203,18 +190,19 @@ private:
}; };
DiskEncrypted::DiskEncrypted( DiskEncrypted::DiskEncrypted(
const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_) const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_)
: DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_)) : DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_), use_fake_transaction_)
{ {
} }
DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_) DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_, bool use_fake_transaction_)
: IDisk(name_) : IDisk(name_)
, delegate(settings_->wrapped_disk) , delegate(settings_->wrapped_disk)
, encrypted_name(name_) , encrypted_name(name_)
, disk_path(settings_->disk_path) , disk_path(settings_->disk_path)
, disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path) , disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path)
, current_settings(std::move(settings_)) , current_settings(std::move(settings_))
, use_fake_transaction(use_fake_transaction_)
{ {
delegate->createDirectories(disk_path); delegate->createDirectories(disk_path);
} }
@ -309,38 +297,6 @@ std::unique_ptr<ReadBufferFromFileBase> DiskEncrypted::readFile(
return std::make_unique<ReadBufferFromEncryptedFile>(settings.local_fs_buffer_size, std::move(buffer), key, header); return std::make_unique<ReadBufferFromEncryptedFile>(settings.local_fs_buffer_size, std::move(buffer), key, header);
} }
std::unique_ptr<WriteBufferFromFileBase> DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &)
{
auto wrapped_path = wrappedPath(path);
FileEncryption::Header header;
String key;
UInt64 old_file_size = 0;
auto settings = current_settings.get();
if (mode == WriteMode::Append && exists(path))
{
old_file_size = getFileSize(path);
if (old_file_size)
{
/// Append mode: we continue to use the same header.
auto read_buffer = delegate->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize));
header = readHeader(*read_buffer);
key = getKey(path, header, *settings);
}
}
if (!old_file_size)
{
/// Rewrite mode: we generate a new header.
key = getCurrentKey(path, *settings);
header.algorithm = settings->current_algorithm;
header.key_id = settings->current_key_id;
header.key_hash = calculateKeyHash(key);
header.init_vector = InitVector::random();
}
auto buffer = delegate->writeFile(wrapped_path, buf_size, mode);
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header, old_file_size);
}
size_t DiskEncrypted::getFileSize(const String & path) const size_t DiskEncrypted::getFileSize(const String & path) const
{ {
auto wrapped_path = wrappedPath(path); auto wrapped_path = wrappedPath(path);
@ -416,7 +372,7 @@ void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check)
const DisksMap & map) -> DiskPtr const DisksMap & map) -> DiskPtr
{ {
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
DiskPtr disk = std::make_shared<DiskEncrypted>(name, config, config_prefix, map); DiskPtr disk = std::make_shared<DiskEncrypted>(name, config, config_prefix, map, config.getBool(config_prefix + ".use_fake_transaction", true));
disk->startup(context, skip_access_check); disk->startup(context, skip_access_check);
return disk; return disk;
}; };

View File

@ -6,22 +6,14 @@
#include <Disks/IDisk.h> #include <Disks/IDisk.h>
#include <Common/MultiVersion.h> #include <Common/MultiVersion.h>
#include <Disks/FakeDiskTransaction.h> #include <Disks/FakeDiskTransaction.h>
#include <Disks/DiskEncryptedTransaction.h>
namespace DB namespace DB
{ {
class ReadBufferFromFileBase; class ReadBufferFromFileBase;
class WriteBufferFromFileBase; class WriteBufferFromFileBase;
namespace FileEncryption { enum class Algorithm; }
struct DiskEncryptedSettings
{
DiskPtr wrapped_disk;
String disk_path;
std::unordered_map<UInt64, String> keys;
UInt64 current_key_id;
FileEncryption::Algorithm current_algorithm;
};
/// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk. /// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk.
/// And when we read files from an encrypted disk it deciphers them automatically, /// And when we read files from an encrypted disk it deciphers them automatically,
@ -29,8 +21,8 @@ struct DiskEncryptedSettings
class DiskEncrypted : public IDisk class DiskEncrypted : public IDisk
{ {
public: public:
DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_); DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_);
DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_); DiskEncrypted(const String & name_, std::unique_ptr<const DiskEncryptedSettings> settings_, bool use_fake_transaction_);
const String & getName() const override { return encrypted_name; } const String & getName() const override { return encrypted_name; }
const String & getPath() const override { return disk_absolute_path; } const String & getPath() const override { return disk_absolute_path; }
@ -59,28 +51,30 @@ public:
void createDirectory(const String & path) override void createDirectory(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->createDirectory(wrapped_path); tx->createDirectory(path);
tx->commit();
} }
void createDirectories(const String & path) override void createDirectories(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->createDirectories(wrapped_path); tx->createDirectories(path);
tx->commit();
} }
void clearDirectory(const String & path) override void clearDirectory(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->clearDirectory(wrapped_path); tx->clearDirectory(path);
tx->commit();
} }
void moveDirectory(const String & from_path, const String & to_path) override void moveDirectory(const String & from_path, const String & to_path) override
{ {
auto wrapped_from_path = wrappedPath(from_path); auto tx = createEncryptedTransaction();
auto wrapped_to_path = wrappedPath(to_path); tx->moveDirectory(from_path, to_path);
delegate->moveDirectory(wrapped_from_path, wrapped_to_path); tx->commit();
} }
DirectoryIteratorPtr iterateDirectory(const String & path) const override DirectoryIteratorPtr iterateDirectory(const String & path) const override
@ -91,22 +85,23 @@ public:
void createFile(const String & path) override void createFile(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->createFile(wrapped_path); tx->createFile(path);
tx->commit();
} }
void moveFile(const String & from_path, const String & to_path) override void moveFile(const String & from_path, const String & to_path) override
{ {
auto wrapped_from_path = wrappedPath(from_path); auto tx = createEncryptedTransaction();
auto wrapped_to_path = wrappedPath(to_path); tx->moveFile(from_path, to_path);
delegate->moveFile(wrapped_from_path, wrapped_to_path); tx->commit();
} }
void replaceFile(const String & from_path, const String & to_path) override void replaceFile(const String & from_path, const String & to_path) override
{ {
auto wrapped_from_path = wrappedPath(from_path); auto tx = createEncryptedTransaction();
auto wrapped_to_path = wrappedPath(to_path); tx->replaceFile(from_path, to_path);
delegate->replaceFile(wrapped_from_path, wrapped_to_path); tx->commit();
} }
void listFiles(const String & path, std::vector<String> & file_names) const override void listFiles(const String & path, std::vector<String> & file_names) const override
@ -129,61 +124,67 @@ public:
const String & path, const String & path,
size_t buf_size, size_t buf_size,
WriteMode mode, WriteMode mode,
const WriteSettings & settings) override; const WriteSettings & settings) override
{
auto tx = createEncryptedTransaction();
auto result = tx->writeFile(path, buf_size, mode, settings);
return result;
}
void removeFile(const String & path) override void removeFile(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeFile(wrapped_path); tx->removeFile(path);
tx->commit();
} }
void removeFileIfExists(const String & path) override void removeFileIfExists(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeFileIfExists(wrapped_path); tx->removeFileIfExists(path);
tx->commit();
} }
void removeDirectory(const String & path) override void removeDirectory(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeDirectory(wrapped_path); tx->removeDirectory(path);
tx->commit();
} }
void removeRecursive(const String & path) override void removeRecursive(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeRecursive(wrapped_path); tx->removeRecursive(path);
tx->commit();
} }
void removeSharedFile(const String & path, bool flag) override void removeSharedFile(const String & path, bool flag) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeSharedFile(wrapped_path, flag); tx->removeSharedFile(path, flag);
tx->commit();
} }
void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeSharedRecursive(wrapped_path, keep_all_batch_data, file_names_remove_metadata_only); tx->removeSharedRecursive(path, keep_all_batch_data, file_names_remove_metadata_only);
tx->commit();
} }
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override
{ {
for (const auto & file : files) auto tx = createEncryptedTransaction();
{ tx->removeSharedFiles(files, keep_all_batch_data, file_names_remove_metadata_only);
auto wrapped_path = wrappedPath(file.path); tx->commit();
bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename());
if (file.if_exists)
delegate->removeSharedFileIfExists(wrapped_path, keep);
else
delegate->removeSharedFile(wrapped_path, keep);
}
} }
void removeSharedFileIfExists(const String & path, bool flag) override void removeSharedFileIfExists(const String & path, bool flag) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->removeSharedFileIfExists(wrapped_path, flag); tx->removeSharedFileIfExists(path, flag);
tx->commit();
} }
Strings getBlobPath(const String & path) const override Strings getBlobPath(const String & path) const override
@ -194,8 +195,9 @@ public:
void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); tx->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function));
tx->commit();
} }
std::unique_ptr<ReadBufferFromFileBase> readEncryptedFile(const String & path, const ReadSettings & settings) const override std::unique_ptr<ReadBufferFromFileBase> readEncryptedFile(const String & path, const ReadSettings & settings) const override
@ -210,8 +212,9 @@ public:
WriteMode mode, WriteMode mode,
const WriteSettings & settings) const override const WriteSettings & settings) const override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
return delegate->writeFile(wrapped_path, buf_size, mode, settings); auto buf = tx->writeEncryptedFile(path, buf_size, mode, settings);
return buf;
} }
size_t getEncryptedFileSize(const String & path) const override size_t getEncryptedFileSize(const String & path) const override
@ -228,8 +231,9 @@ public:
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override void setLastModified(const String & path, const Poco::Timestamp & timestamp) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->setLastModified(wrapped_path, timestamp); tx->setLastModified(path, timestamp);
tx->commit();
} }
Poco::Timestamp getLastModified(const String & path) const override Poco::Timestamp getLastModified(const String & path) const override
@ -246,15 +250,16 @@ public:
void setReadOnly(const String & path) override void setReadOnly(const String & path) override
{ {
auto wrapped_path = wrappedPath(path); auto tx = createEncryptedTransaction();
delegate->setReadOnly(wrapped_path); tx->setReadOnly(path);
tx->commit();
} }
void createHardLink(const String & src_path, const String & dst_path) override void createHardLink(const String & src_path, const String & dst_path) override
{ {
auto wrapped_src_path = wrappedPath(src_path); auto tx = createEncryptedTransaction();
auto wrapped_dst_path = wrappedPath(dst_path); tx->createHardLink(src_path, dst_path);
delegate->createHardLink(wrapped_src_path, wrapped_dst_path); tx->commit();
} }
void truncateFile(const String & path, size_t size) override; void truncateFile(const String & path, size_t size) override;
@ -289,11 +294,22 @@ public:
SyncGuardPtr getDirectorySyncGuard(const String & path) const override; SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
std::shared_ptr<DiskEncryptedTransaction> createEncryptedTransaction() const
{
auto delegate_transaction = delegate->createTransaction();
return std::make_shared<DiskEncryptedTransaction>(delegate_transaction, disk_path, *current_settings.get(), delegate.get());
}
DiskTransactionPtr createTransaction() override DiskTransactionPtr createTransaction() override
{ {
/// Need to overwrite explicetly because this disk change if (use_fake_transaction)
/// a lot of "delegate" methods. {
return std::make_shared<FakeDiskTransaction>(*this); return std::make_shared<FakeDiskTransaction>(*this);
}
else
{
return createEncryptedTransaction();
}
} }
UInt64 getTotalSpace() const override UInt64 getTotalSpace() const override
@ -331,10 +347,7 @@ public:
private: private:
String wrappedPath(const String & path) const String wrappedPath(const String & path) const
{ {
// if path starts_with disk_path -> got already wrapped path return DiskEncryptedTransaction::wrappedPath(disk_path, path);
if (!disk_path.empty() && path.starts_with(disk_path))
return path;
return disk_path + path;
} }
DiskPtr delegate; DiskPtr delegate;
@ -342,6 +355,7 @@ private:
const String disk_path; const String disk_path;
const String disk_absolute_path; const String disk_absolute_path;
MultiVersion<DiskEncryptedSettings> current_settings; MultiVersion<DiskEncryptedSettings> current_settings;
bool use_fake_transaction;
}; };
} }

View File

@ -0,0 +1,120 @@
#include <Disks/DiskEncryptedTransaction.h>
#if USE_SSL
#include <IO/FileEncryptionCommon.h>
#include <Common/Exception.h>
#include <boost/algorithm/hex.hpp>
#include <IO/ReadBufferFromEncryptedFile.h>
#include <IO/ReadBufferFromFileDecorator.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromEncryptedFile.h>
#include <Common/quoteString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int DATA_ENCRYPTION_ERROR;
}
namespace
{
FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer)
{
try
{
FileEncryption::Header header;
header.read(read_buffer);
return header;
}
catch (Exception & e)
{
e.addMessage("While reading the header of encrypted file " + quoteString(read_buffer.getFileName()));
throw;
}
}
String getCurrentKey(const String & path, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(settings.current_key_id);
if (it == settings.keys.end())
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR,
"Not found a key with the current ID {} required to cipher file {}",
settings.current_key_id,
quoteString(path));
return it->second;
}
String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings)
{
auto it = settings.keys.find(header.key_id);
if (it == settings.keys.end())
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR,
"Not found a key with ID {} required to decipher file {}",
header.key_id,
quoteString(path));
String key = it->second;
if (FileEncryption::calculateKeyHash(key) != header.key_hash)
throw Exception(
ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path));
return key;
}
}
void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path)
{
auto wrapped_from_path = wrappedPath(from_file_path);
auto wrapped_to_path = wrappedPath(to_file_path);
delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path);
}
std::unique_ptr<WriteBufferFromFileBase> DiskEncryptedTransaction::writeFile( // NOLINT
const std::string & path,
size_t buf_size,
WriteMode mode,
const WriteSettings & settings,
bool autocommit)
{
auto wrapped_path = wrappedPath(path);
FileEncryption::Header header;
String key;
UInt64 old_file_size = 0;
if (mode == WriteMode::Append && delegate_disk->exists(wrapped_path))
{
size_t size = delegate_disk->getFileSize(wrapped_path);
old_file_size = size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0;
if (old_file_size)
{
/// Append mode: we continue to use the same header.
auto read_buffer = delegate_disk->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize));
header = readHeader(*read_buffer);
key = getKey(path, header, current_settings);
}
}
if (!old_file_size)
{
/// Rewrite mode: we generate a new header.
key = getCurrentKey(path, current_settings);
header.algorithm = current_settings.current_algorithm;
header.key_id = current_settings.current_key_id;
header.key_hash = FileEncryption::calculateKeyHash(key);
header.init_vector = FileEncryption::InitVector::random();
}
auto buffer = delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings, autocommit);
return std::make_unique<WriteBufferFromEncryptedFile>(buf_size, std::move(buffer), key, header, old_file_size);
}
}
#endif

View File

@ -0,0 +1,259 @@
#pragma once
#include "config.h"
#if USE_SSL
#include <Disks/IDiskTransaction.h>
#include <Disks/IDisk.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
namespace DB
{
namespace FileEncryption { enum class Algorithm; }
struct DiskEncryptedSettings
{
DiskPtr wrapped_disk;
String disk_path;
std::unordered_map<UInt64, String> keys;
UInt64 current_key_id;
FileEncryption::Algorithm current_algorithm;
};
class DiskEncryptedTransaction : public IDiskTransaction
{
public:
static String wrappedPath(const String disk_path, const String & path)
{
// if path starts_with disk_path -> got already wrapped path
if (!disk_path.empty() && path.starts_with(disk_path))
return path;
return disk_path + path;
}
DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_)
: delegate_transaction(delegate_transaction_)
, disk_path(disk_path_)
, current_settings(current_settings_)
, delegate_disk(delegate_disk_)
{}
/// Tries to commit all accumulated operations simultaneously.
/// If something fails rollback and throw exception.
void commit() override // NOLINT
{
delegate_transaction->commit();
}
void undo() override
{
delegate_transaction->undo();
}
~DiskEncryptedTransaction() override = default;
/// Create directory.
void createDirectory(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->createDirectory(wrapped_path);
}
/// Create directory and all parent directories if necessary.
void createDirectories(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->createDirectories(wrapped_path);
}
/// Remove all files from the directory. Directories are not removed.
void clearDirectory(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->clearDirectory(wrapped_path);
}
/// Move directory from `from_path` to `to_path`.
void moveDirectory(const std::string & from_path, const std::string & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate_transaction->moveDirectory(wrapped_from_path, wrapped_to_path);
}
void moveFile(const std::string & from_path, const std::string & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate_transaction->moveFile(wrapped_from_path, wrapped_to_path);
}
void createFile(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->createFile(wrapped_path);
}
/// Move the file from `from_path` to `to_path`.
/// If a file with `to_path` path already exists, it will be replaced.
void replaceFile(const std::string & from_path, const std::string & to_path) override
{
auto wrapped_from_path = wrappedPath(from_path);
auto wrapped_to_path = wrappedPath(to_path);
delegate_transaction->replaceFile(wrapped_from_path, wrapped_to_path);
}
/// Only copy of several files supported now. Disk interface support copy to another disk
/// but it's impossible to implement correctly in transactions because other disk can
/// use different metadata storage.
/// TODO: maybe remove it at all, we don't want copies
void copyFile(const std::string & from_file_path, const std::string & to_file_path) override;
/// Open the file for write and return WriteBufferFromFileBase object.
std::unique_ptr<WriteBufferFromFileBase> writeFile( /// NOLINT
const std::string & path,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
WriteMode mode = WriteMode::Rewrite,
const WriteSettings & settings = {},
bool autocommit = true) override;
/// Remove file. Throws exception if file doesn't exists or it's a directory.
void removeFile(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeFile(wrapped_path);
}
/// Remove file if it exists.
void removeFileIfExists(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeFileIfExists(wrapped_path);
}
/// Remove directory. Throws exception if it's not a directory or if directory is not empty.
void removeDirectory(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeDirectory(wrapped_path);
}
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
void removeRecursive(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeRecursive(wrapped_path);
}
/// Remove file. Throws exception if file doesn't exists or if directory is not empty.
/// Differs from removeFile for S3/HDFS disks
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
void removeSharedFile(const std::string & path, bool keep_shared_data) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeSharedFile(wrapped_path, keep_shared_data);
}
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
/// Differs from removeRecursive for S3/HDFS disks
/// Second bool param is a flag to remove (false) or keep (true) shared data on S3.
/// Third param determines which files cannot be removed even if second is true.
void removeSharedRecursive(const std::string & path, bool keep_all_shared_data, const NameSet & file_names_remove_metadata_only) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeSharedRecursive(wrapped_path, keep_all_shared_data, file_names_remove_metadata_only);
}
/// Remove file or directory if it exists.
/// Differs from removeFileIfExists for S3/HDFS disks
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
void removeSharedFileIfExists(const std::string & path, bool keep_shared_data) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->removeSharedFileIfExists(wrapped_path, keep_shared_data);
}
/// Batch request to remove multiple files.
/// May be much faster for blob storage.
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3.
/// Third param determines which files cannot be removed even if second is true.
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override
{
for (const auto & file : files)
{
auto wrapped_path = wrappedPath(file.path);
bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename());
if (file.if_exists)
delegate_transaction->removeSharedFileIfExists(wrapped_path, keep);
else
delegate_transaction->removeSharedFile(wrapped_path, keep);
}
}
/// Set last modified time to file or directory at `path`.
void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->setLastModified(wrapped_path, timestamp);
}
/// Just chmod.
void chmod(const String & path, mode_t mode) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->chmod(wrapped_path, mode);
}
/// Set file at `path` as read-only.
void setReadOnly(const std::string & path) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->setReadOnly(wrapped_path);
}
/// Create hardlink from `src_path` to `dst_path`.
void createHardLink(const std::string & src_path, const std::string & dst_path) override
{
auto wrapped_src_path = wrappedPath(src_path);
auto wrapped_dst_path = wrappedPath(dst_path);
delegate_transaction->createHardLink(wrapped_src_path, wrapped_dst_path);
}
void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override
{
auto wrapped_path = wrappedPath(path);
delegate_transaction->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function));
}
std::unique_ptr<WriteBufferFromFileBase> writeEncryptedFile(
const String & path,
size_t buf_size,
WriteMode mode,
const WriteSettings & settings) const
{
auto wrapped_path = wrappedPath(path);
return delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings);
}
private:
String wrappedPath(const String & path) const
{
return wrappedPath(disk_path, path);
}
DiskTransactionPtr delegate_transaction;
std::string disk_path;
DiskEncryptedSettings current_settings;
IDisk * delegate_disk;
};
}
#endif

View File

@ -40,7 +40,7 @@ protected:
settings->keys[0] = key; settings->keys[0] = key;
settings->current_key_id = 0; settings->current_key_id = 0;
settings->disk_path = path; settings->disk_path = path;
encrypted_disk = std::make_shared<DiskEncrypted>("encrypted_disk", std::move(settings)); encrypted_disk = std::make_shared<DiskEncrypted>("encrypted_disk", std::move(settings), true);
} }
String getFileNames() String getFileNames()

View File

@ -13,7 +13,6 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_LARGE_STRING_SIZE; extern const int TOO_LARGE_STRING_SIZE;
} }
@ -25,18 +24,16 @@ struct RepeatImpl
/// Safety threshold against DoS. /// Safety threshold against DoS.
static inline void checkRepeatTime(UInt64 repeat_time) static inline void checkRepeatTime(UInt64 repeat_time)
{ {
static constexpr UInt64 max_repeat_times = 1000000; static constexpr UInt64 max_repeat_times = 1'000'000;
if (repeat_time > max_repeat_times) if (repeat_time > max_repeat_times)
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times);
std::to_string(repeat_time), std::to_string(max_repeat_times));
} }
static inline void checkStringSize(UInt64 size) static inline void checkStringSize(UInt64 size)
{ {
static constexpr UInt64 max_string_size = 1 << 30; static constexpr UInt64 max_string_size = 1 << 30;
if (size > max_string_size) if (size > max_string_size)
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", size, max_string_size);
size, max_string_size);
} }
template <typename T> template <typename T>
@ -186,36 +183,37 @@ public:
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{ {
if (!isString(arguments[0])) FunctionArgumentDescriptors args{
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", {"s", &isString<IDataType>, nullptr, "String"},
arguments[0]->getName(), getName()); {"n", &isInteger<IDataType>, nullptr, "Integer"},
if (!isInteger(arguments[1])) };
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}",
arguments[1]->getName(), getName()); validateFunctionArgumentTypes(*this, arguments, args);
return arguments[0];
return std::make_shared<DataTypeString>();
} }
bool useDefaultImplementationForConstants() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; }
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override
{ {
const auto & strcolumn = arguments[0].column; const auto & col_str = arguments[0].column;
const auto & numcolumn = arguments[1].column; const auto & col_num = arguments[1].column;
ColumnPtr res; ColumnPtr res;
if (const ColumnString * col = checkAndGetColumn<ColumnString>(strcolumn.get())) if (const ColumnString * col = checkAndGetColumn<ColumnString>(col_str.get()))
{ {
if (const ColumnConst * scale_column_num = checkAndGetColumn<ColumnConst>(numcolumn.get())) if (const ColumnConst * col_num_const = checkAndGetColumn<ColumnConst>(col_num.get()))
{ {
auto col_res = ColumnString::create(); auto col_res = ColumnString::create();
castType(arguments[1].type.get(), [&](const auto & type) castType(arguments[1].type.get(), [&](const auto & type)
{ {
using DataType = std::decay_t<decltype(type)>; using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType; using T = typename DataType::FieldType;
T repeat_time = scale_column_num->getValue<T>(); T times = col_num_const->getValue<T>();
RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time); RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), times);
return true; return true;
}); });
return col_res; return col_res;
@ -224,9 +222,9 @@ public:
{ {
using DataType = std::decay_t<decltype(type)>; using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType; using T = typename DataType::FieldType;
const ColumnVector<T> * colnum = checkAndGetColumn<ColumnVector<T>>(numcolumn.get()); const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(col_num.get());
auto col_res = ColumnString::create(); auto col_res = ColumnString::create();
RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), colnum->getData()); RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData());
res = std::move(col_res); res = std::move(col_res);
return true; return true;
})) }))
@ -234,7 +232,7 @@ public:
return res; return res;
} }
} }
else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(strcolumn.get())) else if (const ColumnConst * col_const = checkAndGetColumn<ColumnConst>(col_str.get()))
{ {
/// Note that const-const case is handled by useDefaultImplementationForConstants. /// Note that const-const case is handled by useDefaultImplementationForConstants.
@ -244,9 +242,9 @@ public:
{ {
using DataType = std::decay_t<decltype(type)>; using DataType = std::decay_t<decltype(type)>;
using T = typename DataType::FieldType; using T = typename DataType::FieldType;
const ColumnVector<T> * colnum = checkAndGetColumn<ColumnVector<T>>(numcolumn.get()); const ColumnVector<T> * column = checkAndGetColumn<ColumnVector<T>>(col_num.get());
auto col_res = ColumnString::create(); auto col_res = ColumnString::create();
RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), colnum->getData()); RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData());
res = std::move(col_res); res = std::move(col_res);
return true; return true;
})) }))

179
src/Functions/space.cpp Normal file
View File

@ -0,0 +1,179 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <cstring>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int TOO_LARGE_STRING_SIZE;
}
namespace
{
/// Prints whitespace n-times. Actually, space() could also be pushed down to repeat(). Chose a standalone-implementation because
/// we can do memset() whereas repeat() does memcpy().
class FunctionSpace : public IFunction
{
private:
static constexpr auto space = ' ';
/// Safety threshold against DoS.
static inline void checkRepeatTime(size_t repeat_time)
{
static constexpr auto max_repeat_times = 1'000'000uz;
if (repeat_time > max_repeat_times)
throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times);
}
public:
static constexpr auto name = "space";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionSpace>(); }
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 1; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
FunctionArgumentDescriptors args{
{"n", &isInteger<IDataType>, nullptr, "Integer"}
};
validateFunctionArgumentTypes(*this, arguments, args);
return std::make_shared<DataTypeString>();
}
template <typename DataType>
bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const
{
const ColumnConst * col_times_const = checkAndGetColumn<ColumnConst>(col_times.get());
const ColumnPtr & col_times_const_internal = col_times_const->getDataColumnPtr();
if (!checkAndGetColumn<typename DataType::ColumnType>(col_times_const_internal.get()))
return false;
using T = typename DataType::FieldType;
T times = col_times_const->getValue<T>();
if (times < 1)
times = 0;
checkRepeatTime(times);
res_offsets.resize(col_times->size());
res_chars.resize(col_times->size() * (times + 1));
size_t pos = 0;
for (size_t i = 0; i < col_times->size(); ++i)
{
memset(res_chars.begin() + pos, space, times);
pos += times;
*(res_chars.begin() + pos) = '\0';
pos += 1;
res_offsets[i] = pos;
}
return true;
}
template <typename DataType>
bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const
{
auto * col_times = checkAndGetColumn<typename DataType::ColumnType>(col_times_.get());
if (!col_times)
return false;
res_offsets.resize(col_times->size());
res_chars.resize(col_times->size() * 10); /// heuristic
const PaddedPODArray<typename DataType::FieldType> & times_data = col_times->getData();
size_t pos = 0;
for (size_t i = 0; i < col_times->size(); ++i)
{
typename DataType::FieldType times = times_data[i];
if (times < 1)
times = 0;
checkRepeatTime(times);
if (pos + times + 1 > res_chars.size())
res_chars.resize(std::max(2 * res_chars.size(), static_cast<size_t>(pos + times + 1)));
memset(res_chars.begin() + pos, space, times);
pos += times;
*(res_chars.begin() + pos) = '\0';
pos += 1;
res_offsets[i] = pos;
}
res_chars.resize(pos);
return true;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
{
const auto & col_num = arguments[0].column;
auto col_res = ColumnString::create();
ColumnString::Offsets & res_offsets = col_res->getOffsets();
ColumnString::Chars & res_chars = col_res->getChars();
if (const ColumnConst * col_num_const = checkAndGetColumn<ColumnConst>(col_num.get()))
{
if ((executeConstant<DataTypeUInt8>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeUInt16>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeUInt32>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeUInt64>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt8>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt16>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt32>(col_num, res_offsets, res_chars))
|| (executeConstant<DataTypeInt64>(col_num, res_offsets, res_chars)))
return col_res;
}
else
{
if ((executeVector<DataTypeUInt8>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeUInt16>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeUInt32>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeUInt64>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt8>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt16>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt32>(col_num, res_offsets, res_chars))
|| (executeVector<DataTypeInt64>(col_num, res_offsets, res_chars)))
return col_res;
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName());
}
};
}
REGISTER_FUNCTION(Space)
{
factory.registerFunction<FunctionSpace>({}, FunctionFactory::CaseInsensitive);
}
}

View File

@ -16,7 +16,7 @@ public:
std::optional<bool> null_modifier; std::optional<bool> null_modifier;
String default_specifier; String default_specifier;
ASTPtr default_expression; ASTPtr default_expression;
bool ephemeral_default; bool ephemeral_default = false;
ASTPtr comment; ASTPtr comment;
ASTPtr codec; ASTPtr codec;
ASTPtr ttl; ASTPtr ttl;

View File

@ -19,13 +19,13 @@ public:
/// Attribute expression /// Attribute expression
ASTPtr expression; ASTPtr expression;
/// Is attribute mirrored to the parent identifier /// Is attribute mirrored to the parent identifier
bool hierarchical; bool hierarchical = false;
/// Is hierarchical attribute bidirectional /// Is hierarchical attribute bidirectional
bool bidirectional; bool bidirectional = false;
/// Flag that shows whether the id->attribute image is injective /// Flag that shows whether the id->attribute image is injective
bool injective; bool injective = false;
/// MongoDB object ID /// MongoDB object ID
bool is_object_id; bool is_object_id = false;
String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); } String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); }

View File

@ -11,14 +11,14 @@ namespace DB
class ASTOrderByElement : public IAST class ASTOrderByElement : public IAST
{ {
public: public:
int direction; /// 1 for ASC, -1 for DESC int direction = 0; /// 1 for ASC, -1 for DESC
int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. int nulls_direction = 0; /// Same as direction for NULLS LAST, opposite for NULLS FIRST.
bool nulls_direction_was_explicitly_specified; bool nulls_direction_was_explicitly_specified = false;
/** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */
ASTPtr collation; ASTPtr collation;
bool with_fill; bool with_fill = false;
ASTPtr fill_from; ASTPtr fill_from;
ASTPtr fill_to; ASTPtr fill_to;
ASTPtr fill_step; ASTPtr fill_step;

View File

@ -35,6 +35,13 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat
{ {
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTO OUTFILE " << (s.hilite ? hilite_none : ""); s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTO OUTFILE " << (s.hilite ? hilite_none : "");
out_file->formatImpl(s, state, frame); out_file->formatImpl(s, state, frame);
s.ostr << (s.hilite ? hilite_keyword : "");
if (is_outfile_append)
s.ostr << " APPEND";
if (is_into_outfile_with_stdout)
s.ostr << " AND STDOUT";
s.ostr << (s.hilite ? hilite_none : "");
} }
if (format) if (format)

View File

@ -15,8 +15,8 @@ class ASTQueryWithOutput : public IAST
{ {
public: public:
ASTPtr out_file; ASTPtr out_file;
bool is_into_outfile_with_stdout; bool is_into_outfile_with_stdout = false;
bool is_outfile_append; bool is_outfile_append = false;
ASTPtr format; ASTPtr format;
ASTPtr settings_ast; ASTPtr settings_ast;
ASTPtr compression; ASTPtr compression;

View File

@ -23,7 +23,7 @@ class ASTWatchQuery : public ASTQueryWithTableAndOutput
public: public:
ASTPtr limit_length; ASTPtr limit_length;
bool is_watch_events; bool is_watch_events = false;
ASTWatchQuery() = default; ASTWatchQuery() = default;
String getID(char) const override { return "WatchQuery_" + getDatabase() + "_" + getTable(); } String getID(char) const override { return "WatchQuery_" + getDatabase() + "_" + getTable(); }

View File

@ -655,6 +655,7 @@ sleep
sleepEachRow sleepEachRow
snowflakeToDateTime snowflakeToDateTime
snowflakeToDateTime64 snowflakeToDateTime64
space
splitByChar splitByChar
splitByNonAlpha splitByNonAlpha
splitByRegexp splitByRegexp

View File

@ -0,0 +1,86 @@
const, uint
3
3
3
3
const, int
3
3
3
3
const, int, negative
0
0
0
0
negative tests
null
\N
const, uint, multiple
const int, multiple
non-const, uint
3
2
1
0
12
10
4
5
4
21
9
7
56
20
5
7
non-const, int
3
2
1
0
12
10
4
5
0
0
0
0
56
20
5
7

View File

@ -0,0 +1,64 @@
SELECT 'const, uint';
SELECT space(3::UInt8), length(space(3::UInt8));
SELECT space(3::UInt16), length(space(3::UInt16));
SELECT space(3::UInt32), length(space(3::UInt32));
SELECT space(3::UInt64), length(space(3::UInt64));
SELECT 'const, int';
SELECT space(3::Int8), length(space(3::Int8));
SELECT space(3::Int16), length(space(3::Int16));
SELECT space(3::Int32), length(space(3::Int32));
SELECT space(3::Int64), length(space(3::Int64));
SELECT 'const, int, negative';
SELECT space(-3::Int8), length(space(-3::Int8));
SELECT space(-3::Int16), length(space(-3::Int16));
SELECT space(-3::Int32), length(space(-3::Int32));
SELECT space(-3::Int64), length(space(-3::Int64));
SELECT 'negative tests';
SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT space(30303030303030303030303030303030::UInt64); -- { serverError TOO_LARGE_STRING_SIZE }
SELECT 'null';
SELECT space(NULL);
DROP TABLE IF EXISTS defaults;
CREATE TABLE defaults
(
u8 UInt8,
u16 UInt16,
u32 UInt32,
u64 UInt64,
i8 Int8,
i16 Int16,
i32 Int32,
i64 Int64
) ENGINE = Memory();
INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7);
SELECT 'const, uint, multiple';
SELECT space(30::UInt8) FROM defaults;
SELECT space(30::UInt16) FROM defaults;
SELECT space(30::UInt32) FROM defaults;
SELECT space(30::UInt64) FROM defaults;
SELECT 'const int, multiple';
SELECT space(30::Int8) FROM defaults;
SELECT space(30::Int16) FROM defaults;
SELECT space(30::Int32) FROM defaults;
SELECT space(30::Int64) FROM defaults;
SELECT 'non-const, uint';
SELECT space(u8), length(space(u8)) FROM defaults;
SELECT space(u16), length(space(u16)) FROM defaults;
SELECT space(u32), length(space(u32)) from defaults;
SELECT space(u64), length(space(u64)) FROM defaults;
SELECT 'non-const, int';
SELECT space(i8), length(space(i8)) FROM defaults;
SELECT space(i16), length(space(i16)) FROM defaults;
SELECT space(i32), length(space(i32)) from defaults;
SELECT space(i64), length(space(i64)) FROM defaults;
DROP TABLE defaults;

View File

@ -0,0 +1,2 @@
Expression ((Projection + Before ORDER BY))
ReadFromStorage (SystemNumbers)

View File

@ -0,0 +1,11 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out"
# only EXPLAIN triggers the problem under MSan
$CLICKHOUSE_CLIENT -q "explain select * from numbers(1) into outfile '$out'"
cat "$out"
rm -f "$out"

View File

@ -0,0 +1,20 @@
SELECT *
FROM numbers(1)
INTO OUTFILE '/dev/null'
;
SELECT *
FROM numbers(1)
INTO OUTFILE '/dev/null' AND STDOUT
;
SELECT *
FROM numbers(1)
INTO OUTFILE '/dev/null' APPEND
;
SELECT *
FROM numbers(1)
INTO OUTFILE '/dev/null' APPEND AND STDOUT
;

View File

@ -0,0 +1,12 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
echo "
select * from numbers(1) into outfile '/dev/null';
select * from numbers(1) into outfile '/dev/null' and stdout;
select * from numbers(1) into outfile '/dev/null' append;
select * from numbers(1) into outfile '/dev/null' append and stdout;
" | clickhouse-format -n